You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2016/03/26 00:19:14 UTC

[7/8] hbase git commit: HBASE-14853 Add on protobuf to c++ chain

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Master.proto b/hbase-native-client/if/Master.proto
new file mode 100644
index 0000000..4d3a2e1
--- /dev/null
+++ b/hbase-native-client/if/Master.proto
@@ -0,0 +1,778 @@
+/**
+ * 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.
+ */
+
+// All to do with the Master.  Includes schema management since these
+// changes are run by the Master process.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "MasterProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "Client.proto";
+import "ClusterStatus.proto";
+import "ErrorHandling.proto";
+import "Procedure.proto";
+import "Quota.proto";
+
+/* Column-level protobufs */
+
+message AddColumnRequest {
+  required TableName table_name = 1;
+  required ColumnFamilySchema column_families = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message AddColumnResponse {
+  optional uint64 proc_id = 1;
+}
+
+message DeleteColumnRequest {
+  required TableName table_name = 1;
+  required bytes column_name = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message DeleteColumnResponse {
+  optional uint64 proc_id = 1;
+}
+
+message ModifyColumnRequest {
+  required TableName table_name = 1;
+  required ColumnFamilySchema column_families = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message ModifyColumnResponse {
+  optional uint64 proc_id = 1;
+}
+
+/* Region-level Protos */
+
+message MoveRegionRequest {
+  required RegionSpecifier region = 1;
+  optional ServerName dest_server_name = 2;
+}
+
+message MoveRegionResponse {
+}
+
+/**
+ * Dispatch merging the specified regions.
+ */
+message DispatchMergingRegionsRequest {
+  required RegionSpecifier region_a = 1;
+  required RegionSpecifier region_b = 2;
+  optional bool forcible = 3 [default = false];
+}
+
+message DispatchMergingRegionsResponse {
+}
+
+message AssignRegionRequest {
+  required RegionSpecifier region = 1;
+}
+
+message AssignRegionResponse {
+}
+
+message UnassignRegionRequest {
+  required RegionSpecifier region = 1;
+  optional bool force = 2 [default = false];
+}
+
+message UnassignRegionResponse {
+}
+
+message OfflineRegionRequest {
+  required RegionSpecifier region = 1;
+}
+
+message OfflineRegionResponse {
+}
+
+/* Table-level protobufs */
+
+message CreateTableRequest {
+  required TableSchema table_schema = 1;
+  repeated bytes split_keys = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message CreateTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message DeleteTableRequest {
+  required TableName table_name = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message DeleteTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message TruncateTableRequest {
+  required TableName tableName = 1;
+  optional bool preserveSplits = 2 [default = false];
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message TruncateTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message EnableTableRequest {
+  required TableName table_name = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message EnableTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message DisableTableRequest {
+  required TableName table_name = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message DisableTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message ModifyTableRequest {
+  required TableName table_name = 1;
+  required TableSchema table_schema = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message ModifyTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+/* Namespace-level protobufs */
+
+message CreateNamespaceRequest {
+  required NamespaceDescriptor namespaceDescriptor = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message CreateNamespaceResponse {
+}
+
+message DeleteNamespaceRequest {
+  required string namespaceName = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message DeleteNamespaceResponse {
+}
+
+message ModifyNamespaceRequest {
+  required NamespaceDescriptor namespaceDescriptor = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message ModifyNamespaceResponse {
+}
+
+message GetNamespaceDescriptorRequest {
+  required string namespaceName = 1;
+}
+
+message GetNamespaceDescriptorResponse {
+  required NamespaceDescriptor namespaceDescriptor = 1;
+}
+
+message ListNamespaceDescriptorsRequest {
+}
+
+message ListNamespaceDescriptorsResponse {
+  repeated NamespaceDescriptor namespaceDescriptor = 1;
+}
+
+message ListTableDescriptorsByNamespaceRequest {
+  required string namespaceName = 1;
+}
+
+message ListTableDescriptorsByNamespaceResponse {
+  repeated TableSchema tableSchema = 1;
+}
+
+message ListTableNamesByNamespaceRequest {
+  required string namespaceName = 1;
+}
+
+message ListTableNamesByNamespaceResponse {
+  repeated TableName tableName = 1;
+}
+
+/* Cluster-level protobufs */
+
+
+message ShutdownRequest {
+}
+
+message ShutdownResponse {
+}
+
+message StopMasterRequest {
+}
+
+message StopMasterResponse {
+}
+
+message BalanceRequest {
+  optional bool force = 1;
+}
+
+message BalanceResponse {
+  required bool balancer_ran = 1;
+}
+
+message SetBalancerRunningRequest {
+  required bool on = 1;
+  optional bool synchronous = 2;
+}
+
+message SetBalancerRunningResponse {
+  optional bool prev_balance_value = 1;
+}
+
+message IsBalancerEnabledRequest {
+}
+
+message IsBalancerEnabledResponse {
+  required bool enabled = 1;
+}
+
+message NormalizeRequest {
+}
+
+message NormalizeResponse {
+  required bool normalizer_ran = 1;
+}
+
+message SetNormalizerRunningRequest {
+  required bool on = 1;
+}
+
+message SetNormalizerRunningResponse {
+  optional bool prev_normalizer_value = 1;
+}
+
+message IsNormalizerEnabledRequest {
+}
+
+message IsNormalizerEnabledResponse {
+  required bool enabled = 1;
+}
+
+message RunCatalogScanRequest {
+}
+
+message RunCatalogScanResponse {
+  optional int32 scan_result = 1;
+}
+
+message EnableCatalogJanitorRequest {
+  required bool enable = 1;
+}
+
+message EnableCatalogJanitorResponse {
+  optional bool prev_value = 1;
+}
+
+message IsCatalogJanitorEnabledRequest {
+}
+
+message IsCatalogJanitorEnabledResponse {
+  required bool value = 1;
+}
+
+message SnapshotRequest {
+	required SnapshotDescription snapshot = 1;
+}
+
+message SnapshotResponse {
+	required int64 expected_timeout = 1;
+}
+
+message GetCompletedSnapshotsRequest {
+}
+
+message GetCompletedSnapshotsResponse {
+	repeated SnapshotDescription snapshots = 1;
+}
+
+message DeleteSnapshotRequest {
+	required SnapshotDescription snapshot = 1;
+}
+
+message DeleteSnapshotResponse {
+}
+
+message RestoreSnapshotRequest {
+  required SnapshotDescription snapshot = 1;
+}
+
+message RestoreSnapshotResponse {
+}
+
+/* if you don't send the snapshot, then you will get it back
+ * in the response (if the snapshot is done) so you can check the snapshot
+ */
+message IsSnapshotDoneRequest {
+	optional SnapshotDescription snapshot = 1;
+}
+
+message IsSnapshotDoneResponse {
+	optional bool done = 1 [default = false];
+	optional SnapshotDescription snapshot = 2;
+}
+
+message IsRestoreSnapshotDoneRequest {
+  optional SnapshotDescription snapshot = 1;
+}
+
+message IsRestoreSnapshotDoneResponse {
+  optional bool done = 1 [default = false];
+}
+
+message GetSchemaAlterStatusRequest {
+  required TableName table_name = 1;
+}
+
+message GetSchemaAlterStatusResponse {
+  optional uint32 yet_to_update_regions = 1;
+  optional uint32 total_regions = 2;
+}
+
+message GetTableDescriptorsRequest {
+  repeated TableName table_names = 1;
+  optional string regex = 2;
+  optional bool include_sys_tables = 3 [default=false];
+  optional string namespace = 4;
+}
+
+message GetTableDescriptorsResponse {
+  repeated TableSchema table_schema = 1;
+}
+
+message GetTableNamesRequest {
+  optional string regex = 1;
+  optional bool include_sys_tables = 2 [default=false];
+  optional string namespace = 3;
+}
+
+message GetTableNamesResponse {
+  repeated TableName table_names = 1;
+}
+
+message GetTableStateRequest {
+  required TableName table_name = 1;
+}
+
+message GetTableStateResponse {
+  required TableState table_state = 1;
+}
+
+message GetClusterStatusRequest {
+}
+
+message GetClusterStatusResponse {
+  required ClusterStatus cluster_status = 1;
+}
+
+message IsMasterRunningRequest {
+}
+
+message IsMasterRunningResponse {
+  required bool is_master_running = 1;
+}
+
+message ExecProcedureRequest {
+  required ProcedureDescription procedure = 1;
+}
+
+message ExecProcedureResponse {
+  optional int64 expected_timeout = 1;
+  optional bytes return_data = 2;
+}
+
+message IsProcedureDoneRequest {
+  optional ProcedureDescription procedure = 1;
+}
+
+message IsProcedureDoneResponse {
+  optional bool done = 1 [default = false];
+  optional ProcedureDescription snapshot = 2;
+}
+
+message GetProcedureResultRequest {
+  required uint64 proc_id = 1;
+}
+
+message GetProcedureResultResponse {
+  enum State {
+    NOT_FOUND = 0;
+    RUNNING = 1;
+    FINISHED = 2;
+  }
+
+  required State state = 1;
+  optional uint64 start_time = 2;
+  optional uint64 last_update = 3;
+  optional bytes result = 4;
+  optional ForeignExceptionMessage exception = 5;
+}
+
+message AbortProcedureRequest {
+  required uint64 proc_id = 1;
+  optional bool mayInterruptIfRunning = 2 [default = true];
+}
+
+message AbortProcedureResponse {
+  required bool is_procedure_aborted = 1;
+}
+
+message ListProceduresRequest {
+}
+
+message ListProceduresResponse {
+  repeated Procedure procedure = 1;
+}
+
+message SetQuotaRequest {
+  optional string user_name = 1;
+  optional string user_group = 2;
+  optional string namespace = 3;
+  optional TableName table_name = 4;
+
+  optional bool remove_all = 5;
+  optional bool bypass_globals = 6;
+  optional ThrottleRequest throttle = 7;
+}
+
+message SetQuotaResponse {
+}
+
+message MajorCompactionTimestampRequest {
+  required TableName table_name = 1;
+}
+
+message MajorCompactionTimestampForRegionRequest {
+  required RegionSpecifier region = 1;
+}
+
+message MajorCompactionTimestampResponse {
+  required int64 compaction_timestamp = 1;
+}
+
+message SecurityCapabilitiesRequest {
+}
+
+message SecurityCapabilitiesResponse {
+  enum Capability {
+    SIMPLE_AUTHENTICATION = 0;
+    SECURE_AUTHENTICATION = 1;
+    AUTHORIZATION = 2;
+    CELL_AUTHORIZATION = 3;
+    CELL_VISIBILITY = 4;
+  }
+
+  repeated Capability capabilities = 1;
+}
+
+service MasterService {
+  /** Used by the client to get the number of regions that have received the updated schema */
+  rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
+    returns(GetSchemaAlterStatusResponse);
+
+  /** Get list of TableDescriptors for requested tables. */
+  rpc GetTableDescriptors(GetTableDescriptorsRequest)
+    returns(GetTableDescriptorsResponse);
+
+  /** Get the list of table names. */
+  rpc GetTableNames(GetTableNamesRequest)
+    returns(GetTableNamesResponse);
+
+  /** Return cluster status. */
+  rpc GetClusterStatus(GetClusterStatusRequest)
+    returns(GetClusterStatusResponse);
+
+  /** return true if master is available */
+  rpc IsMasterRunning(IsMasterRunningRequest) returns(IsMasterRunningResponse);
+
+  /** Adds a column to the specified table. */
+  rpc AddColumn(AddColumnRequest)
+    returns(AddColumnResponse);
+
+  /** Deletes a column from the specified table. Table must be disabled. */
+  rpc DeleteColumn(DeleteColumnRequest)
+    returns(DeleteColumnResponse);
+
+  /** Modifies an existing column on the specified table. */
+  rpc ModifyColumn(ModifyColumnRequest)
+    returns(ModifyColumnResponse);
+
+  /** Move the region region to the destination server. */
+  rpc MoveRegion(MoveRegionRequest)
+    returns(MoveRegionResponse);
+
+ /** Master dispatch merging the regions */
+  rpc DispatchMergingRegions(DispatchMergingRegionsRequest)
+    returns(DispatchMergingRegionsResponse);
+
+  /** Assign a region to a server chosen at random. */
+  rpc AssignRegion(AssignRegionRequest)
+    returns(AssignRegionResponse);
+
+  /**
+   * Unassign a region from current hosting regionserver.  Region will then be
+   * assigned to a regionserver chosen at random.  Region could be reassigned
+   * back to the same server.  Use MoveRegion if you want
+   * to control the region movement.
+   */
+  rpc UnassignRegion(UnassignRegionRequest)
+    returns(UnassignRegionResponse);
+
+  /**
+   * Offline a region from the assignment manager's in-memory state.  The
+   * region should be in a closed state and there will be no attempt to
+   * automatically reassign the region as in unassign.   This is a special
+   * method, and should only be used by experts or hbck.
+   */
+  rpc OfflineRegion(OfflineRegionRequest)
+    returns(OfflineRegionResponse);
+
+  /** Deletes a table */
+  rpc DeleteTable(DeleteTableRequest)
+    returns(DeleteTableResponse);
+
+  /** Truncate a table */
+  rpc truncateTable(TruncateTableRequest)
+    returns(TruncateTableResponse);
+
+  /** Puts the table on-line (only needed if table has been previously taken offline) */
+  rpc EnableTable(EnableTableRequest)
+    returns(EnableTableResponse);
+
+  /** Take table offline */
+  rpc DisableTable(DisableTableRequest)
+    returns(DisableTableResponse);
+
+  /** Modify a table's metadata */
+  rpc ModifyTable(ModifyTableRequest)
+    returns(ModifyTableResponse);
+
+  /** Creates a new table asynchronously */
+  rpc CreateTable(CreateTableRequest)
+    returns(CreateTableResponse);
+
+    /** Shutdown an HBase cluster. */
+  rpc Shutdown(ShutdownRequest)
+    returns(ShutdownResponse);
+
+  /** Stop HBase Master only.  Does not shutdown the cluster. */
+  rpc StopMaster(StopMasterRequest)
+    returns(StopMasterResponse);
+
+  /**
+   * Run the balancer.  Will run the balancer and if regions to move, it will
+   * go ahead and do the reassignments.  Can NOT run for various reasons.
+   * Check logs.
+   */
+  rpc Balance(BalanceRequest)
+    returns(BalanceResponse);
+
+  /**
+   * Turn the load balancer on or off.
+   * If synchronous is true, it waits until current balance() call, if outstanding, to return.
+   */
+  rpc SetBalancerRunning(SetBalancerRunningRequest)
+    returns(SetBalancerRunningResponse);
+
+  /**
+   * Query whether the Region Balancer is running.
+   */
+  rpc IsBalancerEnabled(IsBalancerEnabledRequest)
+    returns(IsBalancerEnabledResponse);
+
+  /**
+   * Run region normalizer. Can NOT run for various reasons. Check logs.
+   */
+  rpc Normalize(NormalizeRequest)
+    returns(NormalizeResponse);
+
+  /**
+   * Turn region normalizer on or off.
+   */
+  rpc SetNormalizerRunning(SetNormalizerRunningRequest)
+    returns(SetNormalizerRunningResponse);
+
+  /**
+   * Query whether region normalizer is enabled.
+   */
+  rpc IsNormalizerEnabled(IsNormalizerEnabledRequest)
+    returns(IsNormalizerEnabledResponse);
+
+  /** Get a run of the catalog janitor */
+  rpc RunCatalogScan(RunCatalogScanRequest)
+     returns(RunCatalogScanResponse);
+
+  /**
+   * Enable the catalog janitor on or off.
+   */
+  rpc EnableCatalogJanitor(EnableCatalogJanitorRequest)
+     returns(EnableCatalogJanitorResponse);
+
+  /**
+   * Query whether the catalog janitor is enabled.
+   */
+  rpc IsCatalogJanitorEnabled(IsCatalogJanitorEnabledRequest)
+     returns(IsCatalogJanitorEnabledResponse);
+
+  /**
+   * Call a master coprocessor endpoint
+   */
+  rpc ExecMasterService(CoprocessorServiceRequest)
+    returns(CoprocessorServiceResponse);
+
+  /**
+   * Create a snapshot for the given table.
+   */
+  rpc Snapshot(SnapshotRequest) returns(SnapshotResponse);
+
+  /**
+   * Get completed snapshots.
+   * Returns a list of snapshot descriptors for completed snapshots
+   */
+  rpc GetCompletedSnapshots(GetCompletedSnapshotsRequest) returns(GetCompletedSnapshotsResponse);
+
+  /**
+   * Delete an existing snapshot. This method can also be used to clean up an aborted snapshot.
+   */
+  rpc DeleteSnapshot(DeleteSnapshotRequest) returns(DeleteSnapshotResponse);
+
+  /**
+   * Determine if the snapshot is done yet.
+   */
+  rpc IsSnapshotDone(IsSnapshotDoneRequest) returns(IsSnapshotDoneResponse);
+
+  /**
+   * Restore a snapshot
+   */
+  rpc RestoreSnapshot(RestoreSnapshotRequest) returns(RestoreSnapshotResponse);
+
+  /**
+   * Determine if the snapshot restore is done yet.
+   */
+  rpc IsRestoreSnapshotDone(IsRestoreSnapshotDoneRequest) returns(IsRestoreSnapshotDoneResponse);
+
+  /**
+   * Execute a distributed procedure.
+   */
+  rpc ExecProcedure(ExecProcedureRequest) returns(ExecProcedureResponse);
+
+  /**
+   * Execute a distributed procedure with return data.
+   */
+  rpc ExecProcedureWithRet(ExecProcedureRequest) returns(ExecProcedureResponse);
+
+  /**
+   * Determine if the procedure is done yet.
+   */
+  rpc IsProcedureDone(IsProcedureDoneRequest) returns(IsProcedureDoneResponse);
+
+  /** return true if master is available */
+  /** rpc IsMasterRunning(IsMasterRunningRequest) returns(IsMasterRunningResponse); */
+
+  /** Modify a namespace's metadata */
+  rpc ModifyNamespace(ModifyNamespaceRequest)
+    returns(ModifyNamespaceResponse);
+
+  /** Creates a new namespace synchronously */
+  rpc CreateNamespace(CreateNamespaceRequest)
+    returns(CreateNamespaceResponse);
+
+  /** Deletes namespace synchronously */
+  rpc DeleteNamespace(DeleteNamespaceRequest)
+    returns(DeleteNamespaceResponse);
+
+  /** Get a namespace descriptor by name */
+  rpc GetNamespaceDescriptor(GetNamespaceDescriptorRequest)
+    returns(GetNamespaceDescriptorResponse);
+
+  /** returns a list of namespaces */
+  rpc ListNamespaceDescriptors(ListNamespaceDescriptorsRequest)
+    returns(ListNamespaceDescriptorsResponse);
+
+  /** returns a list of tables for a given namespace*/
+  rpc ListTableDescriptorsByNamespace(ListTableDescriptorsByNamespaceRequest)
+    returns(ListTableDescriptorsByNamespaceResponse);
+
+  /** returns a list of tables for a given namespace*/
+  rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest)
+    returns(ListTableNamesByNamespaceResponse);
+
+  /** returns table state */
+  rpc GetTableState(GetTableStateRequest)
+    returns(GetTableStateResponse);
+
+  /** Apply the new quota settings */
+  rpc SetQuota(SetQuotaRequest) returns(SetQuotaResponse);
+
+  /** Returns the timestamp of the last major compaction */
+  rpc getLastMajorCompactionTimestamp(MajorCompactionTimestampRequest)
+    returns(MajorCompactionTimestampResponse);
+
+  /** Returns the timestamp of the last major compaction */
+  rpc getLastMajorCompactionTimestampForRegion(MajorCompactionTimestampForRegionRequest)
+    returns(MajorCompactionTimestampResponse);
+
+  rpc getProcedureResult(GetProcedureResultRequest)
+    returns(GetProcedureResultResponse);
+
+  /** Returns the security capabilities in effect on the cluster */
+  rpc getSecurityCapabilities(SecurityCapabilitiesRequest)
+    returns(SecurityCapabilitiesResponse);
+
+  /** Abort a procedure */
+  rpc AbortProcedure(AbortProcedureRequest)
+    returns(AbortProcedureResponse);
+
+  /** returns a list of procedures */
+  rpc ListProcedures(ListProceduresRequest)
+    returns(ListProceduresResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/MasterProcedure.proto b/hbase-native-client/if/MasterProcedure.proto
new file mode 100644
index 0000000..2d2aff4
--- /dev/null
+++ b/hbase-native-client/if/MasterProcedure.proto
@@ -0,0 +1,245 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "MasterProcedureProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "RPC.proto";
+
+// ============================================================================
+//  WARNING - Compatibility rules
+// ============================================================================
+// This .proto contains the data serialized by the master procedures.
+// Each procedure has some state stored to know, which step were executed
+// and what were the parameters or data created by the previous steps.
+// new code should be able to handle the old format or at least fail cleanly
+// triggering a rollback/cleanup.
+//
+// Procedures that are inheriting from a StateMachineProcedure have an enum:
+//  - Do not change the number of the 'State' enums.
+//    doing so, will cause executing the wrong 'step' on the pending
+//    procedures when they will be replayed.
+//  - Do not remove items from the enum, new code must be able to handle
+//    all the previous 'steps'. There may be pending procedure ready to be
+//    recovered replayed. alternative you can make sure that not-known state
+//    will result in a failure that will rollback the already executed steps.
+// ============================================================================
+
+enum CreateTableState {
+  CREATE_TABLE_PRE_OPERATION = 1;
+  CREATE_TABLE_WRITE_FS_LAYOUT = 2;
+  CREATE_TABLE_ADD_TO_META = 3;
+  CREATE_TABLE_ASSIGN_REGIONS = 4;
+  CREATE_TABLE_UPDATE_DESC_CACHE = 5;
+  CREATE_TABLE_POST_OPERATION = 6;
+}
+
+message CreateTableStateData {
+  required UserInformation user_info = 1;
+  required TableSchema table_schema = 2;
+  repeated RegionInfo region_info = 3;
+}
+
+enum ModifyTableState {
+  MODIFY_TABLE_PREPARE = 1;
+  MODIFY_TABLE_PRE_OPERATION = 2;
+  MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR = 3;
+  MODIFY_TABLE_REMOVE_REPLICA_COLUMN = 4;
+  MODIFY_TABLE_DELETE_FS_LAYOUT = 5;
+  MODIFY_TABLE_POST_OPERATION = 6;
+  MODIFY_TABLE_REOPEN_ALL_REGIONS = 7;
+}
+
+message ModifyTableStateData {
+  required UserInformation user_info = 1;
+  optional TableSchema unmodified_table_schema = 2;
+  required TableSchema modified_table_schema = 3;
+  required bool delete_column_family_in_modify = 4;
+}
+
+enum TruncateTableState {
+  TRUNCATE_TABLE_PRE_OPERATION = 1;
+  TRUNCATE_TABLE_REMOVE_FROM_META = 2;
+  TRUNCATE_TABLE_CLEAR_FS_LAYOUT = 3;
+  TRUNCATE_TABLE_CREATE_FS_LAYOUT = 4;
+  TRUNCATE_TABLE_ADD_TO_META = 5;
+  TRUNCATE_TABLE_ASSIGN_REGIONS = 6;
+  TRUNCATE_TABLE_POST_OPERATION = 7;
+}
+
+message TruncateTableStateData {
+  required UserInformation user_info = 1;
+  required bool preserve_splits = 2;
+  optional TableName table_name = 3;
+  optional TableSchema table_schema = 4;
+  repeated RegionInfo region_info = 5;
+}
+
+enum DeleteTableState {
+  DELETE_TABLE_PRE_OPERATION = 1;
+  DELETE_TABLE_REMOVE_FROM_META = 2;
+  DELETE_TABLE_CLEAR_FS_LAYOUT = 3;
+  DELETE_TABLE_UPDATE_DESC_CACHE = 4;
+  DELETE_TABLE_UNASSIGN_REGIONS = 5;
+  DELETE_TABLE_POST_OPERATION = 6;
+}
+
+message DeleteTableStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  repeated RegionInfo region_info = 3;
+}
+
+enum CreateNamespaceState {
+  CREATE_NAMESPACE_PREPARE = 1;
+  CREATE_NAMESPACE_CREATE_DIRECTORY = 2;
+  CREATE_NAMESPACE_INSERT_INTO_NS_TABLE = 3;
+  CREATE_NAMESPACE_UPDATE_ZK = 4;
+  CREATE_NAMESPACE_SET_NAMESPACE_QUOTA = 5;
+}
+
+message CreateNamespaceStateData {
+  required NamespaceDescriptor namespace_descriptor = 1;
+}
+
+enum ModifyNamespaceState {
+  MODIFY_NAMESPACE_PREPARE = 1;
+  MODIFY_NAMESPACE_UPDATE_NS_TABLE = 2;
+  MODIFY_NAMESPACE_UPDATE_ZK = 3;
+}
+
+message ModifyNamespaceStateData {
+  required NamespaceDescriptor namespace_descriptor = 1;
+  optional NamespaceDescriptor unmodified_namespace_descriptor = 2;
+}
+
+enum DeleteNamespaceState {
+  DELETE_NAMESPACE_PREPARE = 1;
+  DELETE_NAMESPACE_DELETE_FROM_NS_TABLE = 2;
+  DELETE_NAMESPACE_REMOVE_FROM_ZK = 3;
+  DELETE_NAMESPACE_DELETE_DIRECTORIES = 4;
+  DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA = 5;
+}
+
+message DeleteNamespaceStateData {
+  required string namespace_name = 1;
+  optional NamespaceDescriptor namespace_descriptor = 2;
+}
+
+enum AddColumnFamilyState {
+  ADD_COLUMN_FAMILY_PREPARE = 1;
+  ADD_COLUMN_FAMILY_PRE_OPERATION = 2;
+  ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  ADD_COLUMN_FAMILY_POST_OPERATION = 4;
+  ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
+}
+
+message AddColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required ColumnFamilySchema columnfamily_schema = 3;
+  optional TableSchema unmodified_table_schema = 4;
+}
+
+enum ModifyColumnFamilyState {
+  MODIFY_COLUMN_FAMILY_PREPARE = 1;
+  MODIFY_COLUMN_FAMILY_PRE_OPERATION = 2;
+  MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  MODIFY_COLUMN_FAMILY_POST_OPERATION = 4;
+  MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
+}
+
+message ModifyColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required ColumnFamilySchema columnfamily_schema = 3;
+  optional TableSchema unmodified_table_schema = 4;
+}
+
+enum DeleteColumnFamilyState {
+  DELETE_COLUMN_FAMILY_PREPARE = 1;
+  DELETE_COLUMN_FAMILY_PRE_OPERATION = 2;
+  DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT = 4;
+  DELETE_COLUMN_FAMILY_POST_OPERATION = 5;
+  DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 6;
+}
+
+message DeleteColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bytes columnfamily_name = 3;
+  optional TableSchema unmodified_table_schema = 4;
+}
+
+enum EnableTableState {
+  ENABLE_TABLE_PREPARE = 1;
+  ENABLE_TABLE_PRE_OPERATION = 2;
+  ENABLE_TABLE_SET_ENABLING_TABLE_STATE = 3;
+  ENABLE_TABLE_MARK_REGIONS_ONLINE = 4;
+  ENABLE_TABLE_SET_ENABLED_TABLE_STATE = 5;
+  ENABLE_TABLE_POST_OPERATION = 6;
+}
+
+message EnableTableStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}
+
+enum DisableTableState {
+  DISABLE_TABLE_PREPARE = 1;
+  DISABLE_TABLE_PRE_OPERATION = 2;
+  DISABLE_TABLE_SET_DISABLING_TABLE_STATE = 3;
+  DISABLE_TABLE_MARK_REGIONS_OFFLINE = 4;
+  DISABLE_TABLE_SET_DISABLED_TABLE_STATE = 5;
+  DISABLE_TABLE_POST_OPERATION = 6;
+}
+
+message DisableTableStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}
+
+message ServerCrashStateData {
+  required ServerName server_name = 1;
+  optional bool distributed_log_replay = 2;
+  repeated RegionInfo regions_on_crashed_server = 3;
+  repeated RegionInfo regions_assigned = 4;
+  optional bool carrying_meta = 5;
+  optional bool should_split_wal = 6 [default = true];
+}
+
+enum ServerCrashState {
+  SERVER_CRASH_START = 1;
+  SERVER_CRASH_PROCESS_META = 2;
+  SERVER_CRASH_GET_REGIONS = 3;
+  SERVER_CRASH_NO_SPLIT_LOGS = 4;
+  SERVER_CRASH_SPLIT_LOGS = 5;
+  SERVER_CRASH_PREPARE_LOG_REPLAY = 6;
+  // Removed SERVER_CRASH_CALC_REGIONS_TO_ASSIGN = 7;
+  SERVER_CRASH_ASSIGN = 8;
+  SERVER_CRASH_WAIT_ON_ASSIGN = 9;
+  SERVER_CRASH_FINISH = 100;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/MultiRowMutation.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/MultiRowMutation.proto b/hbase-native-client/if/MultiRowMutation.proto
new file mode 100644
index 0000000..747afac
--- /dev/null
+++ b/hbase-native-client/if/MultiRowMutation.proto
@@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+import "Client.proto";
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "MultiRowMutationProtos";
+option java_generate_equals_and_hash = true;
+option java_generic_services = true;
+option optimize_for = SPEED;
+
+message MultiRowMutationProcessorRequest{
+}
+
+message MultiRowMutationProcessorResponse{
+}
+
+message MutateRowsRequest {
+  repeated MutationProto mutation_request = 1;
+  optional uint64 nonce_group = 2;
+  optional uint64 nonce = 3;
+}
+
+message MutateRowsResponse {
+}
+
+service MultiRowMutationService {
+  rpc MutateRows(MutateRowsRequest)
+      returns(MutateRowsResponse);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Procedure.proto b/hbase-native-client/if/Procedure.proto
new file mode 100644
index 0000000..55e44a4
--- /dev/null
+++ b/hbase-native-client/if/Procedure.proto
@@ -0,0 +1,119 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ProcedureProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "ErrorHandling.proto";
+
+enum ProcedureState {
+  INITIALIZING = 1;         // Procedure in construction, not yet added to the executor
+  RUNNABLE = 2;             // Procedure added to the executor, and ready to be executed
+  WAITING = 3;              // The procedure is waiting on children to be completed
+  WAITING_TIMEOUT = 4;      // The procedure is waiting a timout or an external event
+  ROLLEDBACK = 5;           // The procedure failed and was rolledback
+  FINISHED = 6;             // The procedure execution is completed. may need a rollback if failed.
+}
+
+/**
+ * Procedure metadata, serialized by the ProcedureStore to be able to recover the old state.
+ */
+message Procedure {
+  // internal "static" state
+  required string class_name = 1;        // full classname to be able to instantiate the procedure
+  optional uint64 parent_id = 2;         // parent if not a root-procedure otherwise not set
+  required uint64 proc_id = 3;
+  required uint64 start_time = 4;
+  optional string owner = 5;
+
+  // internal "runtime" state
+  required ProcedureState state = 6;
+  repeated uint32 stack_id = 7;          // stack indices in case the procedure was running
+  required uint64 last_update = 8;
+  optional uint32 timeout = 9;
+
+  // user state/results
+  optional ForeignExceptionMessage exception = 10;
+  optional bytes result = 11;           // opaque (user) result structure
+  optional bytes state_data = 12;       // opaque (user) procedure internal-state
+
+  // Nonce to prevent same procedure submit by multiple times
+  optional uint64 nonce_group = 13 [default = 0];
+  optional uint64 nonce = 14 [default = 0];
+}
+
+/**
+ * SequentialProcedure data
+ */
+message SequentialProcedureData {
+  required bool executed = 1;
+}
+
+/**
+ * StateMachineProcedure data
+ */
+message StateMachineProcedureData {
+  repeated uint32 state = 1;
+}
+
+/**
+ * Procedure WAL header
+ */
+message ProcedureWALHeader {
+  required uint32 version = 1;
+  required uint32 type = 2;
+  required uint64 log_id = 3;
+  required uint64 min_proc_id = 4;
+}
+
+/**
+ * Procedure WAL trailer
+ */
+message ProcedureWALTrailer {
+  required uint32 version = 1;
+  required uint64 tracker_pos = 2;
+}
+
+message ProcedureStoreTracker {
+  message TrackerNode {
+    required uint64 start_id = 1;
+    repeated uint64 updated = 2;
+    repeated uint64 deleted = 3;
+  }
+
+  repeated TrackerNode node = 1;
+}
+
+message ProcedureWALEntry {
+  enum Type {
+    PROCEDURE_WAL_EOF     = 1;
+    PROCEDURE_WAL_INIT    = 2;
+    PROCEDURE_WAL_INSERT  = 3;
+    PROCEDURE_WAL_UPDATE  = 4;
+    PROCEDURE_WAL_DELETE  = 5;
+    PROCEDURE_WAL_COMPACT = 6;
+  }
+
+  required Type type = 1;
+  repeated Procedure procedure = 2;
+  optional uint64 proc_id = 3;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Quota.proto b/hbase-native-client/if/Quota.proto
new file mode 100644
index 0000000..a8303b1
--- /dev/null
+++ b/hbase-native-client/if/Quota.proto
@@ -0,0 +1,76 @@
+ /**
+ * 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.
+ */
+
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "QuotaProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+enum QuotaScope {
+  CLUSTER = 1;
+  MACHINE = 2;
+}
+
+message TimedQuota {
+  required TimeUnit time_unit = 1;
+  optional uint64 soft_limit  = 2;
+  optional float share = 3;
+  optional QuotaScope scope  = 4 [default = MACHINE];
+}
+
+enum ThrottleType {
+  REQUEST_NUMBER = 1;
+  REQUEST_SIZE   = 2;
+  WRITE_NUMBER   = 3;
+  WRITE_SIZE     = 4;
+  READ_NUMBER    = 5;
+  READ_SIZE      = 6;
+}
+
+message Throttle {
+  optional TimedQuota req_num  = 1;
+  optional TimedQuota req_size = 2;
+
+  optional TimedQuota write_num  = 3;
+  optional TimedQuota write_size = 4;
+
+  optional TimedQuota read_num  = 5;
+  optional TimedQuota read_size = 6;
+}
+
+message ThrottleRequest {
+  optional ThrottleType type = 1;
+  optional TimedQuota timed_quota = 2;
+}
+
+enum QuotaType {
+  THROTTLE = 1;
+}
+
+message Quotas {
+  optional bool bypass_globals = 1 [default = false];
+  optional Throttle throttle = 2;
+}
+
+message QuotaUsage {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/RPC.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RPC.proto b/hbase-native-client/if/RPC.proto
new file mode 100644
index 0000000..59bb03d
--- /dev/null
+++ b/hbase-native-client/if/RPC.proto
@@ -0,0 +1,136 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+import "Tracing.proto"; 
+import "HBase.proto";
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RPCProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+// See https://issues.apache.org/jira/browse/HBASE-7898 for high-level
+// description of RPC specification.
+//
+// On connection setup, the client sends six bytes of preamble -- a four
+// byte magic, a byte of version, and a byte of authentication type.
+//
+// We then send a "ConnectionHeader" protobuf of user information and the
+// 'protocol' or 'service' that is to be run over this connection as well as
+// info such as codecs and compression to use when we send cell blocks(see below).
+// This connection header protobuf is prefaced by an int that holds the length
+// of this connection header (this is NOT a varint).  The pb connection header
+// is sent with Message#writeTo.  The server throws an exception if it doesn't
+// like what it was sent noting what it is objecting too.  Otherwise, the server
+// says nothing and is open for business.
+//
+// Hereafter the client makes requests and the server returns responses.
+//
+// Requests look like this:
+//
+// <An int with the total length of the request>
+// <RequestHeader Message written out using Message#writeDelimitedTo>
+// <Optionally a Request Parameter Message written out using Message#writeDelimitedTo>
+// <Optionally a Cell block>
+//
+// ...where the Request Parameter Message is whatever the method name stipulated
+// in the RequestHeader expects; e.g. if the method is a scan, then the pb
+// Request Message is a GetRequest, or a ScanRequest.  A block of Cells
+// optionally follows.  The presence of a Request param Message and/or a
+// block of Cells will be noted in the RequestHeader.
+//
+// Response is the mirror of the request:
+//
+// <An int with the total length of the response>
+// <ResponseHeader Message written out using Message#writeDelimitedTo>
+// <Optionally a Response Result Message written out using Message#writeDelimitedTo>
+// <Optionally a Cell block>
+//
+// ...where the Response Message is the response type that goes with the
+// method specified when making the request and the follow on Cell blocks may
+// or may not be there -- read the response header to find out if one following.
+// If an exception, it will be included inside the Response Header.
+//
+// Any time we write a pb, we do it with Message#writeDelimitedTo EXCEPT when
+// the connection header is sent; this is prefaced by an int with its length
+// and the pb connection header is then written with Message#writeTo.
+//
+
+// User Information proto.  Included in ConnectionHeader on connection setup
+message UserInformation {
+  required string effective_user = 1;
+  optional string real_user = 2;
+}
+
+// This is sent on connection setup after the connection preamble is sent.
+message ConnectionHeader {
+  optional UserInformation user_info = 1;
+  optional string service_name = 2;
+  // Cell block codec we will use sending over optional cell blocks.  Server throws exception
+  // if cannot deal.  Null means no codec'ing going on so we are pb all the time (SLOW!!!)
+  optional string cell_block_codec_class = 3;
+  // Compressor we will use if cell block is compressed.  Server will throw exception if not supported.
+  // Class must implement hadoop's CompressionCodec Interface.  Can't compress if no codec.
+  optional string cell_block_compressor_class = 4;
+  optional VersionInfo version_info = 5;
+}
+
+// Optional Cell block Message.  Included in client RequestHeader
+message CellBlockMeta {
+  // Length of the following cell block.  Could calculate it but convenient having it too hand.
+  optional uint32 length = 1;
+}
+
+// At the RPC layer, this message is used to carry
+// the server side exception to the RPC client.
+message ExceptionResponse {
+  // Class name of the exception thrown from the server
+  optional string exception_class_name = 1;
+  // Exception stack trace from the server side
+  optional string stack_trace = 2;
+  // Optional hostname.  Filled in for some exceptions such as region moved
+  // where exception gives clue on where the region may have moved.
+  optional string hostname = 3;
+  optional int32 port = 4;
+  // Set if we are NOT to retry on receipt of this exception
+  optional bool do_not_retry = 5;
+}
+
+// Header sent making a request.
+message RequestHeader {
+  // Monotonically increasing call_id to keep track of RPC requests and their response
+  optional uint32 call_id = 1;
+  optional RPCTInfo trace_info = 2;
+  optional string method_name = 3;
+  // If true, then a pb Message param follows.
+  optional bool request_param = 4;
+  // If present, then an encoded data block follows.
+  optional CellBlockMeta cell_block_meta = 5;
+  // 0 is NORMAL priority.  200 is HIGH.  If no priority, treat it as NORMAL.
+  // See HConstants.
+  optional uint32 priority = 6;
+}
+
+message ResponseHeader {
+  optional uint32 call_id = 1;
+  // If present, then request threw an exception and no response message (else we presume one)
+  optional ExceptionResponse exception = 2;
+  // If present, then an encoded data block follows.
+  optional CellBlockMeta cell_block_meta = 3;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/RegionNormalizer.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RegionNormalizer.proto b/hbase-native-client/if/RegionNormalizer.proto
new file mode 100644
index 0000000..e5305d6
--- /dev/null
+++ b/hbase-native-client/if/RegionNormalizer.proto
@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers to represent the state of the load balancer.
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RegionNormalizerProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message RegionNormalizerState {
+    optional bool normalizer_on = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RegionServerStatus.proto b/hbase-native-client/if/RegionServerStatus.proto
new file mode 100644
index 0000000..fda9de2
--- /dev/null
+++ b/hbase-native-client/if/RegionServerStatus.proto
@@ -0,0 +1,158 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used for RegionServerStatusProtocol.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RegionServerStatusProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "ClusterStatus.proto";
+
+message RegionServerStartupRequest {
+  /** Port number this regionserver is up on */
+  required uint32 port = 1;
+
+  /** This servers' startcode */
+  required uint64 server_start_code = 2;
+
+  /** Current time of the region server in ms */
+  required uint64 server_current_time = 3;
+
+  /** hostname for region server, optional */
+  optional string use_this_hostname_instead = 4;
+}
+
+message RegionServerStartupResponse {
+  /**
+   * Configuration for the regionserver to use: e.g. filesystem,
+   * hbase rootdir, the hostname to use creating the RegionServer ServerName,
+   * etc
+   */
+  repeated NameStringPair map_entries = 1;
+}
+
+message RegionServerReportRequest {
+  required ServerName server = 1;
+
+  /** load the server is under */
+  optional ServerLoad load = 2;
+}
+
+message RegionServerReportResponse {
+}
+
+message ReportRSFatalErrorRequest {
+  /** name of the server experiencing the error */
+  required ServerName server = 1;
+
+  /** informative text to expose in the master logs and UI */
+  required string error_message = 2;
+}
+
+message ReportRSFatalErrorResponse {
+}
+
+message GetLastFlushedSequenceIdRequest {
+  /** region name */
+  required bytes region_name = 1;
+}
+
+message GetLastFlushedSequenceIdResponse {
+  /** the last WAL sequence id flushed from MemStore to HFile for the region */
+  required uint64 last_flushed_sequence_id = 1;
+
+  /** the last WAL sequence id flushed from MemStore to HFile for stores of the region */
+  repeated StoreSequenceId store_last_flushed_sequence_id = 2;
+}
+
+message RegionStateTransition {
+  required TransitionCode transition_code = 1;
+
+  /** Mutliple regions are involved during merging/splitting */
+  repeated RegionInfo region_info = 2;
+
+  /** For newly opened region, the open seq num is needed */
+  optional uint64 open_seq_num = 3;
+
+  enum TransitionCode {
+    OPENED = 0;
+    FAILED_OPEN = 1;
+    /** No failed_close, in which case region server will abort */
+    CLOSED = 2;
+
+    /** Ask master for ok to split/merge region(s) */
+    READY_TO_SPLIT = 3;
+    READY_TO_MERGE = 4;
+
+    SPLIT_PONR = 5;
+    MERGE_PONR = 6;
+
+    SPLIT = 7;
+    MERGED = 8;
+    SPLIT_REVERTED = 9;
+    MERGE_REVERTED = 10;
+  }
+}
+
+message ReportRegionStateTransitionRequest {
+  /** This region server's server name */
+  required ServerName server = 1;
+
+  repeated RegionStateTransition transition = 2;
+}
+
+message ReportRegionStateTransitionResponse {
+  /** Error message if failed to update the region state */
+  optional string error_message = 1;
+}
+
+service RegionServerStatusService {
+  /** Called when a region server first starts. */
+  rpc RegionServerStartup(RegionServerStartupRequest)
+    returns(RegionServerStartupResponse);
+
+  /** Called to report the load the RegionServer is under. */
+  rpc RegionServerReport(RegionServerReportRequest)
+    returns(RegionServerReportResponse);
+
+  /**
+   * Called by a region server to report a fatal error that is causing it to
+   * abort.
+   */
+  rpc ReportRSFatalError(ReportRSFatalErrorRequest)
+    returns(ReportRSFatalErrorResponse);
+
+  /** Called to get the sequence id of the last MemStore entry flushed to an
+   * HFile for a specified region. Used by the region server to speed up
+   * log splitting. */
+  rpc GetLastFlushedSequenceId(GetLastFlushedSequenceIdRequest)
+    returns(GetLastFlushedSequenceIdResponse);
+
+  /**
+   * Called by a region server to report the progress of a region
+   * transition. If the request fails, the transition should
+   * be aborted.
+   */
+  rpc ReportRegionStateTransition(ReportRegionStateTransitionRequest)
+    returns(ReportRegionStateTransitionResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/RowProcessor.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RowProcessor.proto b/hbase-native-client/if/RowProcessor.proto
new file mode 100644
index 0000000..cf2f30f
--- /dev/null
+++ b/hbase-native-client/if/RowProcessor.proto
@@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+/**
+ * Defines a protocol to perform multi row transactions.
+ * See BaseRowProcessorEndpoint for the implementation.
+ * See HRegion#processRowsWithLocks() for details.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RowProcessorProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message ProcessRequest {
+  required string row_processor_class_name = 1;
+  optional string row_processor_initializer_message_name = 2;
+  optional bytes  row_processor_initializer_message = 3;
+  optional uint64 nonce_group = 4;
+  optional uint64 nonce = 5;
+}
+
+message ProcessResponse {
+  required bytes row_processor_result = 1;
+}
+
+service RowProcessorService {
+  rpc Process(ProcessRequest) returns (ProcessResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/SecureBulkLoad.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/SecureBulkLoad.proto b/hbase-native-client/if/SecureBulkLoad.proto
new file mode 100644
index 0000000..814735b
--- /dev/null
+++ b/hbase-native-client/if/SecureBulkLoad.proto
@@ -0,0 +1,72 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "SecureBulkLoadProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import 'HBase.proto';
+import 'Client.proto';
+
+message SecureBulkLoadHFilesRequest {
+  repeated BulkLoadHFileRequest.FamilyPath family_path = 1;
+  optional bool assign_seq_num = 2;
+  required DelegationToken fs_token = 3;
+  required string bulk_token = 4;
+}
+
+message SecureBulkLoadHFilesResponse {
+  required bool loaded = 1;
+}
+
+message DelegationToken {
+  optional bytes identifier = 1;
+  optional bytes password = 2;
+  optional string kind = 3;
+  optional string service = 4;
+}
+
+message PrepareBulkLoadRequest {
+  required TableName table_name = 1;
+}
+
+message PrepareBulkLoadResponse {
+  required string bulk_token = 1;
+}
+
+message CleanupBulkLoadRequest {
+  required string bulk_token = 1;
+
+}
+
+message CleanupBulkLoadResponse {
+}
+
+service SecureBulkLoadService {
+    rpc PrepareBulkLoad(PrepareBulkLoadRequest)
+      returns (PrepareBulkLoadResponse);
+
+    rpc SecureBulkLoadHFiles(SecureBulkLoadHFilesRequest)
+      returns (SecureBulkLoadHFilesResponse);
+
+    rpc CleanupBulkLoad(CleanupBulkLoadRequest)
+      returns (CleanupBulkLoadResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/Snapshot.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Snapshot.proto b/hbase-native-client/if/Snapshot.proto
new file mode 100644
index 0000000..ae1a1e6
--- /dev/null
+++ b/hbase-native-client/if/Snapshot.proto
@@ -0,0 +1,66 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "SnapshotProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "FS.proto";
+import "HBase.proto";
+
+message SnapshotFileInfo {
+  enum Type {
+    HFILE = 1;
+    WAL = 2;
+  }
+
+  required Type type = 1;
+
+  optional string hfile = 3;
+
+  optional string wal_server = 4;
+  optional string wal_name = 5;
+}
+
+message SnapshotRegionManifest {
+  optional int32 version = 1;
+
+  required RegionInfo region_info = 2;
+  repeated FamilyFiles family_files = 3;
+
+  message StoreFile {
+    required string name = 1;
+    optional Reference reference = 2;
+
+    // TODO: Add checksums or other fields to verify the file
+    optional uint64 file_size = 3;
+  }
+
+  message FamilyFiles {
+    required bytes family_name = 1;
+    repeated StoreFile store_files = 2;
+  }
+}
+
+message SnapshotDataManifest {
+  required TableSchema table_schema = 1;
+  repeated SnapshotRegionManifest region_manifests = 2;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/Tracing.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Tracing.proto b/hbase-native-client/if/Tracing.proto
new file mode 100644
index 0000000..5a64cfc
--- /dev/null
+++ b/hbase-native-client/if/Tracing.proto
@@ -0,0 +1,33 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "TracingProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+//Used to pass through the information necessary to continue
+//a trace after an RPC is made. All we need is the traceid 
+//(so we know the overarching trace this message is a part of), and
+//the id of the current span when this message was sent, so we know 
+//what span caused the new span we will create when this message is received.
+message RPCTInfo {
+  optional int64 trace_id = 1;
+  optional int64 parent_id = 2;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/VisibilityLabels.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/VisibilityLabels.proto b/hbase-native-client/if/VisibilityLabels.proto
new file mode 100644
index 0000000..d2dc44d
--- /dev/null
+++ b/hbase-native-client/if/VisibilityLabels.proto
@@ -0,0 +1,83 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "VisibilityLabelsProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "Client.proto";
+
+message VisibilityLabelsRequest {
+  repeated VisibilityLabel visLabel = 1;
+}
+
+message VisibilityLabel {
+  required bytes label = 1;
+  optional uint32 ordinal = 2;
+}
+
+message VisibilityLabelsResponse {
+  repeated RegionActionResult result = 1; 
+}
+
+message SetAuthsRequest {
+  required bytes user = 1;
+  repeated bytes auth = 2;
+}
+
+message UserAuthorizations {
+  required bytes user = 1;
+  repeated uint32 auth = 2;
+}
+
+message MultiUserAuthorizations {
+  repeated UserAuthorizations userAuths = 1;
+}
+
+message GetAuthsRequest {
+  required bytes user = 1;
+}
+
+message GetAuthsResponse {
+  required bytes user = 1;
+  repeated bytes auth = 2;
+}
+
+message ListLabelsRequest {
+  optional string regex = 1;
+}
+
+message ListLabelsResponse {
+  repeated bytes label = 1;
+}
+
+service VisibilityLabelsService {
+  rpc addLabels(VisibilityLabelsRequest)
+    returns (VisibilityLabelsResponse);
+  rpc setAuths(SetAuthsRequest)
+    returns (VisibilityLabelsResponse);
+  rpc clearAuths(SetAuthsRequest)
+    returns (VisibilityLabelsResponse);
+  rpc getAuths(GetAuthsRequest)
+    returns (GetAuthsResponse);
+  rpc listLabels(ListLabelsRequest)
+    returns (ListLabelsResponse);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/WAL.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/WAL.proto b/hbase-native-client/if/WAL.proto
new file mode 100644
index 0000000..cb9bd8f
--- /dev/null
+++ b/hbase-native-client/if/WAL.proto
@@ -0,0 +1,172 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "WALProtos";
+option java_generic_services = false;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "Client.proto";
+
+message WALHeader {
+  optional bool has_compression = 1;
+  optional bytes encryption_key = 2;
+  optional bool has_tag_compression = 3;
+  optional string writer_cls_name = 4;
+  optional string cell_codec_cls_name = 5;
+}
+
+/*
+ * Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header
+ * for some KVs
+ */
+message WALKey {
+  required bytes encoded_region_name = 1;
+  required bytes table_name = 2;
+  required uint64 log_sequence_number = 3;
+  required uint64 write_time = 4;
+  /*
+  This parameter is deprecated in favor of clusters which 
+  contains the list of clusters that have consumed the change.
+  It is retained so that the log created by earlier releases (0.94) 
+  can be read by the newer releases.
+  */
+  optional UUID cluster_id = 5 [deprecated=true];
+
+  repeated FamilyScope scopes = 6;
+  optional uint32 following_kv_count = 7;
+
+  /*
+  This field contains the list of clusters that have
+  consumed the change
+  */
+  repeated UUID cluster_ids = 8;
+
+  optional uint64 nonceGroup = 9;
+  optional uint64 nonce = 10;
+  optional uint64 orig_sequence_number = 11;
+
+/*
+  optional CustomEntryType custom_entry_type = 9;
+
+  enum CustomEntryType {
+    COMPACTION = 0;
+  }
+*/
+}
+
+enum ScopeType {
+  REPLICATION_SCOPE_LOCAL = 0;
+  REPLICATION_SCOPE_GLOBAL = 1;
+}
+
+message FamilyScope {
+  required bytes family = 1;
+  required ScopeType scope_type = 2;
+}
+
+/**
+ * Custom WAL entries
+ */
+
+/**
+ * Special WAL entry to hold all related to a compaction.
+ * Written to WAL before completing compaction.  There is
+ * sufficient info in the below message to complete later
+ * the * compaction should we fail the WAL write.
+ */
+message CompactionDescriptor {
+  required bytes table_name = 1; // TODO: WALKey already stores these, might remove
+  required bytes encoded_region_name = 2;
+  required bytes family_name = 3;
+  repeated string compaction_input = 4; // relative to store dir
+  repeated string compaction_output = 5;
+  required string store_home_dir = 6; // relative to region dir
+  optional bytes  region_name = 7; // full region name
+}
+
+/**
+ * Special WAL entry to hold all related to a flush.
+ */
+message FlushDescriptor {
+  enum FlushAction {
+    START_FLUSH = 0;
+    COMMIT_FLUSH = 1;
+    ABORT_FLUSH = 2;
+    CANNOT_FLUSH = 3; // marker for indicating that a flush has been requested but cannot complete
+  }
+
+  message StoreFlushDescriptor {
+    required bytes family_name = 1;
+    required string store_home_dir = 2; //relative to region dir
+    repeated string flush_output = 3; // relative to store dir (if this is a COMMIT_FLUSH)
+  }
+
+  required FlushAction action = 1;
+  required bytes table_name = 2;
+  required bytes encoded_region_name = 3;
+  optional uint64 flush_sequence_number = 4;
+  repeated StoreFlushDescriptor store_flushes = 5;
+  optional bytes  region_name = 6; // full region name
+}
+
+message StoreDescriptor {
+  required bytes family_name = 1;
+  required string store_home_dir = 2; //relative to region dir
+  repeated string store_file = 3; // relative to store dir
+}
+
+/**
+ * Special WAL entry used for writing bulk load events to WAL
+ */
+message BulkLoadDescriptor {
+  required TableName table_name = 1;
+  required bytes encoded_region_name = 2;
+  repeated StoreDescriptor stores = 3;
+  required int64 bulkload_seq_num = 4;
+}
+
+/**
+ * Special WAL entry to hold all related to a region event (open/close).
+ */
+message RegionEventDescriptor {
+  enum EventType {
+    REGION_OPEN = 0;
+    REGION_CLOSE = 1;
+  }
+
+  required EventType event_type = 1;
+  required bytes table_name = 2;
+  required bytes encoded_region_name = 3;
+  optional uint64 log_sequence_number = 4;
+  repeated StoreDescriptor stores = 5;
+  optional ServerName server = 6;  // Server who opened the region
+  optional bytes  region_name = 7; // full region name
+}
+
+/**
+ * A trailer that is appended to the end of a properly closed WAL file.
+ * If missing, this is either a legacy or a corrupted WAL file.
+ * N.B. This trailer currently doesn't contain any information and we
+ * purposefully don't expose it in the WAL APIs. It's for future growth.
+ */
+message WALTrailer {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/if/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/ZooKeeper.proto b/hbase-native-client/if/ZooKeeper.proto
new file mode 100644
index 0000000..54652af
--- /dev/null
+++ b/hbase-native-client/if/ZooKeeper.proto
@@ -0,0 +1,155 @@
+/**
+ * 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.
+ */
+
+// ZNode data in hbase are serialized protobufs with a four byte
+// 'magic' 'PBUF' prefix.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ZooKeeperProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "ClusterStatus.proto";
+
+/**
+ * Content of the meta-region-server znode.
+ */
+message MetaRegionServer {
+  // The ServerName hosting the meta region currently, or destination server,
+  // if meta region is in transition.
+  required ServerName server = 1;
+  // The major version of the rpc the server speaks.  This is used so that
+  // clients connecting to the cluster can have prior knowledge of what version
+  // to send to a RegionServer.  AsyncHBase will use this to detect versions.
+  optional uint32 rpc_version = 2;
+
+  // State of the region transition. OPEN means fully operational 'hbase:meta'
+  optional RegionState.State state = 3;
+}
+
+/**
+ * Content of the master znode.
+ */
+message Master {
+  // The ServerName of the current Master
+  required ServerName master = 1;
+  // Major RPC version so that clients can know what version the master can accept.
+  optional uint32 rpc_version = 2;
+  optional uint32 info_port = 3;
+}
+
+/**
+ * Content of the '/hbase/running', cluster state, znode.
+ */
+message ClusterUp {
+  // If this znode is present, cluster is up.  Currently
+  // the data is cluster start_date.
+  required string start_date = 1;
+}
+
+/**
+ * WAL SplitLog directory znodes have this for content.  Used doing distributed
+ * WAL splitting.  Holds current state and name of server that originated split.
+ */
+message SplitLogTask {
+  enum State {
+    UNASSIGNED = 0;
+    OWNED = 1;
+    RESIGNED = 2;
+    DONE = 3;
+    ERR = 4;
+  }
+  enum RecoveryMode {
+    UNKNOWN = 0;
+    LOG_SPLITTING = 1;
+    LOG_REPLAY = 2;
+  }
+  required State state = 1;
+  required ServerName server_name = 2;
+  optional RecoveryMode mode = 3 [default = UNKNOWN];
+}
+
+/**
+ * The znode that holds state of table.
+ * Deprected, table state is stored in table descriptor on HDFS.
+ */
+message DeprecatedTableState {
+  // Table's current state
+  enum State {
+    ENABLED = 0;
+    DISABLED = 1;
+    DISABLING = 2;
+    ENABLING = 3;
+  }
+  // This is the table's state.  If no znode for a table,
+  // its state is presumed enabled.  See o.a.h.h.zookeeper.ZKTable class
+  // for more.
+  required State state = 1 [default = ENABLED];
+}
+
+/**
+ * Used by replication. Holds a replication peer key.
+ */
+message ReplicationPeer {
+  // clusterkey is the concatenation of the slave cluster's
+  // hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+  required string clusterkey = 1;
+  optional string replicationEndpointImpl = 2;
+  repeated BytesBytesPair data = 3;
+  repeated NameStringPair configuration = 4;
+}
+
+/**
+ * Used by replication. Holds whether enabled or disabled
+ */
+message ReplicationState {
+  enum State {
+    ENABLED = 0;
+    DISABLED = 1;
+  }
+  required State state = 1;
+}
+
+/**
+ * Used by replication. Holds the current position in an WAL file.
+ */
+message ReplicationHLogPosition {
+  required int64 position = 1;
+}
+
+/**
+ * Used by replication. Used to lock a region server during failover.
+ */
+message ReplicationLock {
+  required string lock_owner = 1;
+}
+
+/**
+ * Metadata associated with a table lock in zookeeper
+ */
+message TableLock {
+  optional TableName table_name = 1;
+  optional ServerName lock_owner = 2;
+  optional int64 thread_id = 3;
+  optional bool is_shared = 4;
+  optional string purpose = 5;
+  optional int64 create_time = 6;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/rpc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hbase-native-client/rpc/CMakeLists.txt b/hbase-native-client/rpc/CMakeLists.txt
deleted file mode 100644
index 2456923..0000000
--- a/hbase-native-client/rpc/CMakeLists.txt
+++ /dev/null
@@ -1,17 +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.
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/967f0910/hbase-native-client/third-party/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/third-party/BUCK b/hbase-native-client/third-party/BUCK
index 89d2c93..b7baa86 100644
--- a/hbase-native-client/third-party/BUCK
+++ b/hbase-native-client/third-party/BUCK
@@ -21,7 +21,8 @@ def add_system_libs(names = []):
                 prebuilt_cxx_library(
                         name = name,
                         lib_name = name,
-                        lib_dir = "/usr/lib/x86_64-linux-gnu"
+                        lib_dir = "/usr/lib/x86_64-linux-gnu",
+                        visibility = [ 'PUBLIC', ],
                 )
                 rules.append(":" + name)
 
@@ -30,6 +31,7 @@ def add_system_libs(names = []):
 system_libs = [
         "double-conversion",
         "glog",
+        "protobuf",
         "gflags",
         "unwind",
         "lzma",