- * Without this protobufs would force us to copy every single byte array out
- * of the objects de-serialized from the wire (which already do one copy, on
- * top of the copies the JVM does to go from kernel buffer to C buffer and
- * from C buffer to JVM buffer).
- *
- * @since 0.96.1
- */
-@InterfaceAudience.Private
-public final class HBaseZeroCopyByteString extends LiteralByteString {
- // Gotten from AsyncHBase code base with permission.
- /** Private constructor so this class cannot be instantiated. */
- private HBaseZeroCopyByteString() {
- super(null);
- throw new UnsupportedOperationException("Should never be here.");
- }
-
- /**
- * Wraps a byte array in a {@link ByteString} without copying it.
- * @param array array to be wrapped
- * @return wrapped array
- */
- public static ByteString wrap(final byte[] array) {
- return new LiteralByteString(array);
- }
-
- /**
- * Wraps a subset of a byte array in a {@link ByteString} without copying it.
- * @param array array to be wrapped
- * @param offset from
- * @param length length
- * @return wrapped array
- */
- public static ByteString wrap(final byte[] array, int offset, int length) {
- return new BoundedByteString(array, offset, length);
- }
-
- // TODO:
- // ZeroCopyLiteralByteString.wrap(this.buf, 0, this.count);
-
- /**
- * Extracts the byte array from the given {@link ByteString} without copy.
- * @param buf A buffer from which to extract the array. This buffer must be
- * actually an instance of a {@code LiteralByteString}.
- * @return byte[] representation
- */
- public static byte[] zeroCopyGetBytes(final ByteString buf) {
- if (buf instanceof LiteralByteString) {
- return ((LiteralByteString) buf).bytes;
- }
- throw new UnsupportedOperationException("Need a LiteralByteString, got a "
- + buf.getClass().getName());
- }
-}
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/util/ByteStringer.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/util/ByteStringer.java
deleted file mode 100644
index 581741d835b..00000000000
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/util/ByteStringer.java
+++ /dev/null
@@ -1,69 +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.
- */
-package org.apache.hadoop.hbase.util;
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.HBaseZeroCopyByteString;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Hack to workaround HBASE-10304 issue that keeps bubbling up when a mapreduce context.
- */
-@InterfaceAudience.Private
-public final class ByteStringer {
- private static final Logger LOG = LoggerFactory.getLogger(ByteStringer.class);
-
- /**
- * Flag set at class loading time.
- */
- private static boolean USE_ZEROCOPYBYTESTRING = true;
-
- // Can I classload HBaseZeroCopyByteString without IllegalAccessError?
- // If we can, use it passing ByteStrings to pb else use native ByteString though more costly
- // because it makes a copy of the passed in array.
- static {
- try {
- HBaseZeroCopyByteString.wrap(new byte [0]);
- } catch (IllegalAccessError iae) {
- USE_ZEROCOPYBYTESTRING = false;
- LOG.debug("Failed to classload HBaseZeroCopyByteString: " + iae.toString());
- }
- }
-
- private ByteStringer() {
- super();
- }
-
- /**
- * Wraps a byte array in a {@link ByteString} without copying it.
- */
- public static ByteString wrap(final byte[] array) {
- return USE_ZEROCOPYBYTESTRING? HBaseZeroCopyByteString.wrap(array): ByteString.copyFrom(array);
- }
-
- /**
- * Wraps a subset of a byte array in a {@link ByteString} without copying it.
- */
- public static ByteString wrap(final byte[] array, int offset, int length) {
- return USE_ZEROCOPYBYTESTRING? HBaseZeroCopyByteString.wrap(array, offset, length):
- ByteString.copyFrom(array, offset, length);
- }
-}
diff --git a/hbase-protocol/src/main/protobuf/AccessControl.proto b/hbase-protocol/src/main/protobuf/AccessControl.proto
deleted file mode 100644
index c35bb5bfbae..00000000000
--- a/hbase-protocol/src/main/protobuf/AccessControl.proto
+++ /dev/null
@@ -1,140 +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.
- */
-syntax = "proto2";
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "AccessControlProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-message Permission {
- enum Action {
- READ = 0;
- WRITE = 1;
- EXEC = 2;
- CREATE = 3;
- ADMIN = 4;
- }
- enum Type {
- Global = 1;
- Namespace = 2;
- Table = 3;
- }
- required Type type = 1;
- optional GlobalPermission global_permission = 2;
- optional NamespacePermission namespace_permission = 3;
- optional TablePermission table_permission = 4;
-}
-
-message TablePermission {
- optional TableName table_name = 1;
- optional bytes family = 2;
- optional bytes qualifier = 3;
- repeated Permission.Action action = 4;
-}
-
-message NamespacePermission {
- optional bytes namespace_name = 1;
- repeated Permission.Action action = 2;
-}
-
-message GlobalPermission {
- repeated Permission.Action action = 1;
-}
-
-message UserPermission {
- required bytes user = 1;
- required Permission permission = 3;
-}
-
-/**
- * Content of the /hbase/acl/
znode.
- */
-message UsersAndPermissions {
- message UserPermissions {
- required bytes user = 1;
- repeated Permission permissions = 2;
- }
-
- repeated UserPermissions user_permissions = 1;
-}
-
-message GrantRequest {
- required UserPermission user_permission = 1;
- optional bool merge_existing_permissions = 2 [default = false];
-}
-
-message GrantResponse {
-}
-
-message RevokeRequest {
- required UserPermission user_permission = 1;
-}
-
-message RevokeResponse {
-}
-
-message GetUserPermissionsRequest {
- optional Permission.Type type = 1;
- optional TableName table_name = 2;
- optional bytes namespace_name = 3;
- optional bytes column_family = 4;
- optional bytes column_qualifier = 5;
- optional bytes user_name = 6;
-}
-
-message GetUserPermissionsResponse {
- repeated UserPermission user_permission = 1;
-}
-
-message CheckPermissionsRequest {
- repeated Permission permission = 1;
-}
-
-message CheckPermissionsResponse {
-}
-
-message HasPermissionRequest {
- required TablePermission table_permission = 1;
- required bytes user_name = 2;
-}
-
-message HasPermissionResponse {
- optional bool has_permission = 1;
-}
-
-service AccessControlService {
- rpc Grant(GrantRequest)
- returns (GrantResponse);
-
- rpc Revoke(RevokeRequest)
- returns (RevokeResponse);
-
- rpc GetUserPermissions(GetUserPermissionsRequest)
- returns (GetUserPermissionsResponse);
-
- rpc CheckPermissions(CheckPermissionsRequest)
- returns (CheckPermissionsResponse);
-
- rpc HasPermission(HasPermissionRequest)
- returns (HasPermissionResponse);
-}
diff --git a/hbase-protocol/src/main/protobuf/Admin.proto b/hbase-protocol/src/main/protobuf/Admin.proto
deleted file mode 100644
index 68194d63b1b..00000000000
--- a/hbase-protocol/src/main/protobuf/Admin.proto
+++ /dev/null
@@ -1,310 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers that are used for Admin service.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "AdminProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "WAL.proto";
-
-message GetRegionInfoRequest {
- required RegionSpecifier region = 1;
- optional bool compaction_state = 2;
-}
-
-message GetRegionInfoResponse {
- required RegionInfo region_info = 1;
- optional CompactionState compaction_state = 2;
- // optional bool DEPRECATED_isRecovering = 3;
-
- enum CompactionState {
- NONE = 0;
- MINOR = 1;
- MAJOR = 2;
- MAJOR_AND_MINOR = 3;
- }
-}
-
-/**
- * Get a list of store files for a set of column families in a particular region.
- * If no column family is specified, get the store files for all column families.
- */
-message GetStoreFileRequest {
- required RegionSpecifier region = 1;
- repeated bytes family = 2;
-}
-
-message GetStoreFileResponse {
- repeated string store_file = 1;
-}
-
-message GetOnlineRegionRequest {
-}
-
-message GetOnlineRegionResponse {
- repeated RegionInfo region_info = 1;
-}
-
-message OpenRegionRequest {
- repeated RegionOpenInfo open_info = 1;
- // the intended server for this RPC.
- optional uint64 serverStartCode = 2;
- // wall clock time from master
- optional uint64 master_system_time = 5;
-
- message RegionOpenInfo {
- required RegionInfo region = 1;
- optional uint32 version_of_offline_node = 2;
- repeated ServerName favored_nodes = 3;
- // open region for distributedLogReplay
- // optional bool DEPRECATED_openForDistributedLogReplay = 4;
- }
-}
-
-message OpenRegionResponse {
- repeated RegionOpeningState opening_state = 1;
-
- enum RegionOpeningState {
- OPENED = 0;
- ALREADY_OPENED = 1;
- FAILED_OPENING = 2;
- }
-}
-
-message WarmupRegionRequest {
-
- required RegionInfo regionInfo = 1;
-}
-
-message WarmupRegionResponse {
-}
-
-/**
- * Closes the specified region and will use or not use ZK during the close
- * according to the specified flag.
- */
-message CloseRegionRequest {
- required RegionSpecifier region = 1;
- optional uint32 version_of_closing_node = 2;
- optional bool transition_in_ZK = 3 [default = true];
- optional ServerName destination_server = 4;
- // the intended server for this RPC.
- optional uint64 serverStartCode = 5;
-}
-
-message CloseRegionResponse {
- required bool closed = 1;
-}
-
-/**
- * Flushes the MemStore of the specified region.
- *
- * This method is synchronous.
- */
-message FlushRegionRequest {
- required RegionSpecifier region = 1;
- optional uint64 if_older_than_ts = 2;
- optional bool write_flush_wal_marker = 3; // whether to write a marker to WAL even if not flushed
-}
-
-message FlushRegionResponse {
- required uint64 last_flush_time = 1;
- optional bool flushed = 2;
- optional bool wrote_flush_wal_marker = 3;
-}
-
-/**
- * Splits the specified region.
- *
- * This method currently flushes the region and then forces a compaction which
- * will then trigger a split. The flush is done synchronously but the
- * compaction is asynchronous.
- */
-message SplitRegionRequest {
- required RegionSpecifier region = 1;
- optional bytes split_point = 2;
-}
-
-message SplitRegionResponse {
-}
-
-/**
- * Compacts the specified region. Performs a major compaction if specified.
- *
- * This method is asynchronous.
- */
-message CompactRegionRequest {
- required RegionSpecifier region = 1;
- optional bool major = 2;
- optional bytes family = 3;
-}
-
-message CompactRegionResponse {
-}
-
-message UpdateFavoredNodesRequest {
- repeated RegionUpdateInfo update_info = 1;
-
- message RegionUpdateInfo {
- required RegionInfo region = 1;
- repeated ServerName favored_nodes = 2;
- }
-}
-
-message UpdateFavoredNodesResponse {
- optional uint32 response = 1;
-}
-
-/**
- * Merges the specified regions.
- *
- * This method currently closes the regions and then merges them
- */
-message MergeRegionsRequest {
- required RegionSpecifier region_a = 1;
- required RegionSpecifier region_b = 2;
- optional bool forcible = 3 [default = false];
- // wall clock time from master
- optional uint64 master_system_time = 4;
-}
-
-message MergeRegionsResponse {
-}
-
-// Protocol buffer version of WAL for replication
-message WALEntry {
- required WALKey key = 1;
- // Following may be null if the KVs/Cells are carried along the side in a cellblock (See
- // RPC for more on cellblocks). If Cells/KVs are in a cellblock, this next field is null
- // and associated_cell_count has count of Cells associated w/ this WALEntry
- repeated bytes key_value_bytes = 2;
- // If Cell data is carried alongside in a cellblock, this is count of Cells in the cellblock.
- optional int32 associated_cell_count = 3;
-}
-
-/**
- * Replicates the given entries. The guarantee is that the given entries
- * will be durable on the slave cluster if this method returns without
- * any exception.
- */
-message ReplicateWALEntryRequest {
- repeated WALEntry entry = 1;
- optional string replicationClusterId = 2;
- optional string sourceBaseNamespaceDirPath = 3;
- optional string sourceHFileArchiveDirPath = 4;
-}
-
-message ReplicateWALEntryResponse {
-}
-
-message RollWALWriterRequest {
-}
-
-/*
- * Roll request responses no longer include regions to flush
- * this list will always be empty when talking to a 1.0 server
- */
-message RollWALWriterResponse {
- // A list of encoded name of regions to flush
- repeated bytes region_to_flush = 1;
-}
-
-message StopServerRequest {
- required string reason = 1;
-}
-
-message StopServerResponse {
-}
-
-message GetServerInfoRequest {
-}
-
-message ServerInfo {
- required ServerName server_name = 1;
- optional uint32 webui_port = 2;
-}
-
-message GetServerInfoResponse {
- required ServerInfo server_info = 1;
-}
-
-message UpdateConfigurationRequest {
-}
-
-message UpdateConfigurationResponse {
-}
-
-service AdminService {
- rpc GetRegionInfo(GetRegionInfoRequest)
- returns(GetRegionInfoResponse);
-
- rpc GetStoreFile(GetStoreFileRequest)
- returns(GetStoreFileResponse);
-
- rpc GetOnlineRegion(GetOnlineRegionRequest)
- returns(GetOnlineRegionResponse);
-
- rpc OpenRegion(OpenRegionRequest)
- returns(OpenRegionResponse);
-
- rpc WarmupRegion(WarmupRegionRequest)
- returns(WarmupRegionResponse);
-
- rpc CloseRegion(CloseRegionRequest)
- returns(CloseRegionResponse);
-
- rpc FlushRegion(FlushRegionRequest)
- returns(FlushRegionResponse);
-
- rpc SplitRegion(SplitRegionRequest)
- returns(SplitRegionResponse);
-
- rpc CompactRegion(CompactRegionRequest)
- returns(CompactRegionResponse);
-
- rpc MergeRegions(MergeRegionsRequest)
- returns(MergeRegionsResponse);
-
- rpc ReplicateWALEntry(ReplicateWALEntryRequest)
- returns(ReplicateWALEntryResponse);
-
- rpc Replay(ReplicateWALEntryRequest)
- returns(ReplicateWALEntryResponse);
-
- rpc RollWALWriter(RollWALWriterRequest)
- returns(RollWALWriterResponse);
-
- rpc GetServerInfo(GetServerInfoRequest)
- returns(GetServerInfoResponse);
-
- rpc StopServer(StopServerRequest)
- returns(StopServerResponse);
-
- rpc UpdateFavoredNodes(UpdateFavoredNodesRequest)
- returns(UpdateFavoredNodesResponse);
-
- rpc UpdateConfiguration(UpdateConfigurationRequest)
- returns(UpdateConfigurationResponse);
-}
diff --git a/hbase-protocol/src/main/protobuf/Authentication.proto b/hbase-protocol/src/main/protobuf/Authentication.proto
deleted file mode 100644
index aa0211f7b48..00000000000
--- a/hbase-protocol/src/main/protobuf/Authentication.proto
+++ /dev/null
@@ -1,83 +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.
- */
-syntax = "proto2";
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "AuthenticationProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-message AuthenticationKey {
- required int32 id = 1;
- required int64 expiration_date = 2;
- required bytes key = 3;
-}
-
-
-message TokenIdentifier {
- enum Kind {
- HBASE_AUTH_TOKEN = 0;
- }
- required Kind kind = 1;
- required bytes username = 2;
- required int32 key_id = 3;
- optional int64 issue_date = 4;
- optional int64 expiration_date = 5;
- optional int64 sequence_number = 6;
-}
-
-
-// Serialization of the org.apache.hadoop.security.token.Token class
-// Note that this is a Hadoop class, so fields may change!
-message Token {
- // the TokenIdentifier in serialized form
- // Note: we can't use the protobuf directly because the Hadoop Token class
- // only stores the serialized bytes
- optional bytes identifier = 1;
- optional bytes password = 2;
- optional bytes service = 3;
-}
-
-
-// RPC request & response messages
-message GetAuthenticationTokenRequest {
-}
-
-message GetAuthenticationTokenResponse {
- optional Token token = 1;
-}
-
-message WhoAmIRequest {
-}
-
-message WhoAmIResponse {
- optional string username = 1;
- optional string auth_method = 2;
-}
-
-
-// RPC service
-service AuthenticationService {
- rpc GetAuthenticationToken(GetAuthenticationTokenRequest)
- returns (GetAuthenticationTokenResponse);
-
- rpc WhoAmI(WhoAmIRequest)
- returns (WhoAmIResponse);
-}
diff --git a/hbase-protocol/src/main/protobuf/Cell.proto b/hbase-protocol/src/main/protobuf/Cell.proto
deleted file mode 100644
index f9628081897..00000000000
--- a/hbase-protocol/src/main/protobuf/Cell.proto
+++ /dev/null
@@ -1,69 +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.
- */
-syntax = "proto2";
-
-// Cell and KeyValue protos
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "CellProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-/**
- * The type of the key in a Cell
- */
-enum CellType {
- MINIMUM = 0;
- PUT = 4;
-
- DELETE = 8;
- DELETE_FAMILY_VERSION = 10;
- DELETE_COLUMN = 12;
- DELETE_FAMILY = 14;
-
- // MAXIMUM is used when searching; you look from maximum on down.
- MAXIMUM = 255;
-}
-
-/**
- * Protocol buffer version of Cell.
- */
-message Cell {
- optional bytes row = 1;
- optional bytes family = 2;
- optional bytes qualifier = 3;
- optional uint64 timestamp = 4;
- optional CellType cell_type = 5;
- optional bytes value = 6;
- optional bytes tags = 7;
-}
-
-/**
- * Protocol buffer version of KeyValue.
- * It doesn't have those transient parameters
- */
-message KeyValue {
- required bytes row = 1;
- required bytes family = 2;
- required bytes qualifier = 3;
- optional uint64 timestamp = 4;
- optional CellType key_type = 5;
- optional bytes value = 6;
- optional bytes tags = 7;
-}
diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto
deleted file mode 100644
index a8454f922d2..00000000000
--- a/hbase-protocol/src/main/protobuf/Client.proto
+++ /dev/null
@@ -1,550 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers that are used for Client service.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ClientProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "Filter.proto";
-import "Cell.proto";
-import "Comparator.proto";
-import "MapReduce.proto";
-
-/**
- * The protocol buffer version of Authorizations.
- */
-message Authorizations {
- repeated string label = 1;
-}
-
-/**
- * The protocol buffer version of CellVisibility.
- */
-message CellVisibility {
- required string expression = 1;
-}
-
-/**
- * Container for a list of column qualifier names of a family.
- */
-message Column {
- required bytes family = 1;
- repeated bytes qualifier = 2;
-}
-
-/**
- * Consistency defines the expected consistency level for an operation.
- */
-enum Consistency {
- STRONG = 0;
- TIMELINE = 1;
-}
-
-/**
- * The protocol buffer version of Get.
- * Unless existence_only is specified, return all the requested data
- * for the row that matches exactly.
- */
-message Get {
- required bytes row = 1;
- repeated Column column = 2;
- repeated NameBytesPair attribute = 3;
- optional Filter filter = 4;
- optional TimeRange time_range = 5;
- optional uint32 max_versions = 6 [default = 1];
- optional bool cache_blocks = 7 [default = true];
- optional uint32 store_limit = 8;
- optional uint32 store_offset = 9;
-
- // The result isn't asked for, just check for
- // the existence.
- optional bool existence_only = 10 [default = false];
-
- // If the row to get doesn't exist, return the
- // closest row before. Deprecated. No longer used!
- // Since hbase-2.0.0.
- optional bool closest_row_before = 11 [default = false];
-
- optional Consistency consistency = 12 [default = STRONG];
- repeated ColumnFamilyTimeRange cf_time_range = 13;
- optional bool load_column_families_on_demand = 14; /* DO NOT add defaults to load_column_families_on_demand. */
-}
-
-message Result {
- // Result includes the Cells or else it just has a count of Cells
- // that are carried otherwise.
- repeated Cell cell = 1;
- // The below count is set when the associated cells are
- // not part of this protobuf message; they are passed alongside
- // and then this Message is just a placeholder with metadata.
- // The count is needed to know how many to peel off the block of Cells as
- // ours. NOTE: This is different from the pb managed cell_count of the
- // 'cell' field above which is non-null when the cells are pb'd.
- optional int32 associated_cell_count = 2;
-
- // used for Get to check existence only. Not set if existence_only was not set to true
- // in the query.
- optional bool exists = 3;
-
- // Whether or not the results are coming from possibly stale data
- optional bool stale = 4 [default = false];
-
- // Whether or not the entire result could be returned. Results will be split when
- // the RPC chunk size limit is reached. Partial results contain only a subset of the
- // cells for a row and must be combined with a result containing the remaining cells
- // to form a complete result. The equivalent flag in o.a.h.h.client.Result is
- // mayHaveMoreCellsInRow.
- optional bool partial = 5 [default = false];
-}
-
-/**
- * The get request. Perform a single Get operation.
- */
-message GetRequest {
- required RegionSpecifier region = 1;
- required Get get = 2;
-}
-
-message GetResponse {
- optional Result result = 1;
-}
-
-/**
- * Condition to check if the value of a given cell (row,
- * family, qualifier) matches a value via a given comparator.
- *
- * Condition is used in check and mutate operations.
- */
-message Condition {
- required bytes row = 1;
- optional bytes family = 2;
- optional bytes qualifier = 3;
- optional CompareType compare_type = 4;
- optional Comparator comparator = 5;
- optional TimeRange time_range = 6;
- optional Filter filter = 7;
-}
-
-
-/**
- * A specific mutation inside a mutate request.
- * It can be an append, increment, put or delete based
- * on the mutation type. It can be fully filled in or
- * only metadata present because data is being carried
- * elsewhere outside of pb.
- */
-message MutationProto {
- optional bytes row = 1;
- optional MutationType mutate_type = 2;
- repeated ColumnValue column_value = 3;
- optional uint64 timestamp = 4;
- repeated NameBytesPair attribute = 5;
- optional Durability durability = 6 [default = USE_DEFAULT];
-
- // For some mutations, a result may be returned, in which case,
- // time range can be specified for potential performance gain
- optional TimeRange time_range = 7;
- // The below count is set when the associated cells are NOT
- // part of this protobuf message; they are passed alongside
- // and then this Message is a placeholder with metadata. The
- // count is needed to know how many to peel off the block of Cells as
- // ours. NOTE: This is different from the pb managed cell_count of the
- // 'cell' field above which is non-null when the cells are pb'd.
- optional int32 associated_cell_count = 8;
-
- optional uint64 nonce = 9;
-
- enum Durability {
- USE_DEFAULT = 0;
- SKIP_WAL = 1;
- ASYNC_WAL = 2;
- SYNC_WAL = 3;
- FSYNC_WAL = 4;
- }
-
- enum MutationType {
- APPEND = 0;
- INCREMENT = 1;
- PUT = 2;
- DELETE = 3;
- }
-
- enum DeleteType {
- DELETE_ONE_VERSION = 0;
- DELETE_MULTIPLE_VERSIONS = 1;
- DELETE_FAMILY = 2;
- DELETE_FAMILY_VERSION = 3;
- }
-
- message ColumnValue {
- required bytes family = 1;
- repeated QualifierValue qualifier_value = 2;
-
- message QualifierValue {
- optional bytes qualifier = 1;
- optional bytes value = 2;
- optional uint64 timestamp = 3;
- optional DeleteType delete_type = 4;
- optional bytes tags = 5;
- }
- }
-}
-
-/**
- * The mutate request. Perform a single Mutate operation.
- *
- * Optionally, you can specify a condition. The mutate
- * will take place only if the condition is met. Otherwise,
- * the mutate will be ignored. In the response result,
- * parameter processed is used to indicate if the mutate
- * actually happened.
- */
-message MutateRequest {
- required RegionSpecifier region = 1;
- required MutationProto mutation = 2;
- optional Condition condition = 3;
- optional uint64 nonce_group = 4;
-}
-
-message MutateResponse {
- optional Result result = 1;
-
- // used for mutate to indicate processed only
- optional bool processed = 2;
-}
-
-/**
- * Instead of get from a table, you can scan it with optional filters.
- * You can specify the row key range, time range, the columns/families
- * to scan and so on.
- *
- * This scan is used the first time in a scan request. The response of
- * the initial scan will return a scanner id, which should be used to
- * fetch result batches later on before it is closed.
- */
-message Scan {
- repeated Column column = 1;
- repeated NameBytesPair attribute = 2;
- optional bytes start_row = 3;
- optional bytes stop_row = 4;
- optional Filter filter = 5;
- optional TimeRange time_range = 6;
- optional uint32 max_versions = 7 [default = 1];
- optional bool cache_blocks = 8 [default = true];
- optional uint32 batch_size = 9;
- optional uint64 max_result_size = 10;
- optional uint32 store_limit = 11;
- optional uint32 store_offset = 12;
- optional bool load_column_families_on_demand = 13; /* DO NOT add defaults to load_column_families_on_demand. */
- optional bool small = 14 [deprecated = true];
- optional bool reversed = 15 [default = false];
- optional Consistency consistency = 16 [default = STRONG];
- optional uint32 caching = 17;
- optional bool allow_partial_results = 18;
- repeated ColumnFamilyTimeRange cf_time_range = 19;
- optional uint64 mvcc_read_point = 20 [default = 0];
- optional bool include_start_row = 21 [default = true];
- optional bool include_stop_row = 22 [default = false];
- enum ReadType {
- DEFAULT = 0;
- STREAM = 1;
- PREAD = 2;
- }
- optional ReadType readType = 23 [default = DEFAULT];
- optional bool need_cursor_result = 24 [default = false];
-}
-
-/**
- * A scan request. Initially, it should specify a scan. Later on, you
- * can use the scanner id returned to fetch result batches with a different
- * scan request.
- *
- * The scanner will remain open if there are more results, and it's not
- * asked to be closed explicitly.
- *
- * You can fetch the results and ask the scanner to be closed to save
- * a trip if you are not interested in remaining results.
- */
-message ScanRequest {
- optional RegionSpecifier region = 1;
- optional Scan scan = 2;
- optional uint64 scanner_id = 3;
- optional uint32 number_of_rows = 4;
- optional bool close_scanner = 5;
- optional uint64 next_call_seq = 6;
- optional bool client_handles_partials = 7;
- optional bool client_handles_heartbeats = 8;
- optional bool track_scan_metrics = 9;
- optional bool renew = 10 [default = false];
- // if we have returned limit_of_rows rows to client, then close the scanner.
- optional uint32 limit_of_rows = 11 [default = 0];
-}
-
-/**
-* Scan cursor to tell client where we are scanning.
-*
- */
-message Cursor {
- optional bytes row = 1;
-}
-
-/**
- * The scan response. If there are no more results, more_results will
- * be false. If it is not specified, it means there are more.
- */
-message ScanResponse {
- // This field is filled in if we are doing cellblocks. A cellblock is made up
- // of all Cells serialized out as one cellblock BUT responses from a server
- // have their Cells grouped by Result. So we can reconstitute the
- // Results on the client-side, this field is a list of counts of Cells
- // in each Result that makes up the response. For example, if this field
- // has 3, 3, 3 in it, then we know that on the client, we are to make
- // three Results each of three Cells each.
- repeated uint32 cells_per_result = 1;
-
- optional uint64 scanner_id = 2;
- optional bool more_results = 3;
- optional uint32 ttl = 4;
- // If cells are not carried in an accompanying cellblock, then they are pb'd here.
- // This field is mutually exclusive with cells_per_result (since the Cells will
- // be inside the pb'd Result)
- repeated Result results = 5;
- optional bool stale = 6;
-
- // This field is filled in if we are doing cellblocks. In the event that a row
- // could not fit all of its cells into a single RPC chunk, the results will be
- // returned as partials, and reconstructed into a complete result on the client
- // side. This field is a list of flags indicating whether or not the result
- // that the cells belong to is a partial result. For example, if this field
- // has false, false, true in it, then we know that on the client side, we need to
- // make another RPC request since the last result was only a partial.
- repeated bool partial_flag_per_result = 7;
-
- // A server may choose to limit the number of results returned to the client for
- // reasons such as the size in bytes or quantity of results accumulated. This field
- // will true when more results exist in the current region.
- optional bool more_results_in_region = 8;
-
- // This field is filled in if the server is sending back a heartbeat message.
- // Heartbeat messages are sent back to the client to prevent the scanner from
- // timing out. Seeing a heartbeat message communicates to the Client that the
- // server would have continued to scan had the time limit not been reached.
- optional bool heartbeat_message = 9;
-
- // This field is filled in if the client has requested that scan metrics be tracked.
- // The metrics tracked here are sent back to the client to be tracked together with
- // the existing client side metrics.
- optional ScanMetrics scan_metrics = 10;
-
- // The mvcc read point which is used to open the scanner at server side. Client can
- // make use of this mvcc_read_point when restarting a scanner to get a consistent view
- // of a row.
- optional uint64 mvcc_read_point = 11 [default = 0];
-
- // If the Scan need cursor, return the row key we are scanning in heartbeat message.
- // If the Scan doesn't need a cursor, don't set this field to reduce network IO.
- optional Cursor cursor = 12;
-}
-
-/**
- * Atomically bulk load multiple HFiles (say from different column families)
- * into an open region.
- */
-message BulkLoadHFileRequest {
- required RegionSpecifier region = 1;
- repeated FamilyPath family_path = 2;
- optional bool assign_seq_num = 3;
- optional DelegationToken fs_token = 4;
- optional string bulk_token = 5;
- optional bool copy_file = 6 [default = false];
-
- message FamilyPath {
- required bytes family = 1;
- required string path = 2;
- }
-}
-
-message BulkLoadHFileResponse {
- 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;
- optional RegionSpecifier region = 2;
-}
-
-message PrepareBulkLoadResponse {
- required string bulk_token = 1;
-}
-
-message CleanupBulkLoadRequest {
- required string bulk_token = 1;
- optional RegionSpecifier region = 2;
-}
-
-message CleanupBulkLoadResponse {
-}
-
-message CoprocessorServiceCall {
- required bytes row = 1;
- required string service_name = 2;
- required string method_name = 3;
- required bytes request = 4;
-}
-
-message CoprocessorServiceResult {
- optional NameBytesPair value = 1;
-}
-
-message CoprocessorServiceRequest {
- required RegionSpecifier region = 1;
- required CoprocessorServiceCall call = 2;
-}
-
-message CoprocessorServiceResponse {
- required RegionSpecifier region = 1;
- required NameBytesPair value = 2;
-}
-
-// Either a Get or a Mutation
-message Action {
- // If part of a multi action, useful aligning
- // result with what was originally submitted.
- optional uint32 index = 1;
- optional MutationProto mutation = 2;
- optional Get get = 3;
- optional CoprocessorServiceCall service_call = 4;
-}
-
-/**
- * Actions to run against a Region.
- */
-message RegionAction {
- required RegionSpecifier region = 1;
- // When set, run mutations as atomic unit.
- optional bool atomic = 2;
- repeated Action action = 3;
-}
-
-/*
-* Statistics about the current load on the region
-*/
-message RegionLoadStats {
- // Percent load on the memstore. Guaranteed to be positive, between 0 and 100.
- optional int32 memStoreLoad = 1 [default = 0];
- // Percent JVM heap occupancy. Guaranteed to be positive, between 0 and 100.
- // We can move this to "ServerLoadStats" should we develop them.
- optional int32 heapOccupancy = 2 [default = 0];
- // Compaction pressure. Guaranteed to be positive, between 0 and 100.
- optional int32 compactionPressure = 3 [default = 0];
-}
-
-message MultiRegionLoadStats{
- repeated RegionSpecifier region = 1;
- repeated RegionLoadStats stat = 2;
-}
-
-/**
- * Either a Result or an Exception NameBytesPair (keyed by
- * exception name whose value is the exception stringified)
- * or maybe empty if no result and no exception.
- */
-message ResultOrException {
- // If part of a multi call, save original index of the list of all
- // passed so can align this response w/ original request.
- optional uint32 index = 1;
- optional Result result = 2;
- optional NameBytesPair exception = 3;
- // result if this was a coprocessor service call
- optional CoprocessorServiceResult service_result = 4;
- // current load on the region
- optional RegionLoadStats loadStats = 5 [deprecated=true];
-}
-
-/**
- * The result of a RegionAction.
- */
-message RegionActionResult {
- repeated ResultOrException resultOrException = 1;
- // If the operation failed globally for this region, this exception is set
- optional NameBytesPair exception = 2;
-}
-
-/**
- * Execute a list of actions on a given region in order.
- * Nothing prevents a request to contains a set of RegionAction on the same region.
- * For this reason, the matching between the MultiRequest and the MultiResponse is not
- * done by the region specifier but by keeping the order of the RegionActionResult vs.
- * the order of the RegionAction.
- */
-message MultiRequest {
- repeated RegionAction regionAction = 1;
- optional uint64 nonceGroup = 2;
- optional Condition condition = 3;
-}
-
-message MultiResponse {
- repeated RegionActionResult regionActionResult = 1;
- // used for mutate to indicate processed only
- optional bool processed = 2;
- optional MultiRegionLoadStats regionStatistics = 3;
-}
-
-
-service ClientService {
- rpc Get(GetRequest)
- returns(GetResponse);
-
- rpc Mutate(MutateRequest)
- returns(MutateResponse);
-
- rpc Scan(ScanRequest)
- returns(ScanResponse);
-
- rpc BulkLoadHFile(BulkLoadHFileRequest)
- returns(BulkLoadHFileResponse);
-
- rpc PrepareBulkLoad(PrepareBulkLoadRequest)
- returns (PrepareBulkLoadResponse);
-
- rpc CleanupBulkLoad(CleanupBulkLoadRequest)
- returns (CleanupBulkLoadResponse);
-
- rpc ExecService(CoprocessorServiceRequest)
- returns(CoprocessorServiceResponse);
-
- rpc ExecRegionServerService(CoprocessorServiceRequest)
- returns(CoprocessorServiceResponse);
-
- rpc Multi(MultiRequest)
- returns(MultiResponse);
-}
diff --git a/hbase-protocol/src/main/protobuf/ClusterId.proto b/hbase-protocol/src/main/protobuf/ClusterId.proto
deleted file mode 100644
index 8d916a690fd..00000000000
--- a/hbase-protocol/src/main/protobuf/ClusterId.proto
+++ /dev/null
@@ -1,35 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers that are shared throughout HBase
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ClusterIdProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-/**
- * Content of the '/hbase/hbaseid', cluster id, znode.
- * Also cluster of the ${HBASE_ROOTDIR}/hbase.id file.
- */
-message ClusterId {
- // This is the cluster id, a uuid as a String
- required string cluster_id = 1;
-}
diff --git a/hbase-protocol/src/main/protobuf/ClusterStatus.proto b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
deleted file mode 100644
index a41ac7113c4..00000000000
--- a/hbase-protocol/src/main/protobuf/ClusterStatus.proto
+++ /dev/null
@@ -1,283 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers that are used for ClustStatus
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ClusterStatusProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "ClusterId.proto";
-import "FS.proto";
-
-message RegionState {
- required RegionInfo region_info = 1;
- required State state = 2;
- optional uint64 stamp = 3;
- enum State {
- OFFLINE = 0; // region is in an offline state
- PENDING_OPEN = 1; // sent rpc to server to open but has not begun
- OPENING = 2; // server has begun to open but not yet done
- OPEN = 3; // server opened region and updated meta
- PENDING_CLOSE = 4; // sent rpc to server to close but has not begun
- CLOSING = 5; // server has begun to close but not yet done
- CLOSED = 6; // server closed region and updated meta
- SPLITTING = 7; // server started split of a region
- SPLIT = 8; // server completed split of a region
- FAILED_OPEN = 9; // failed to open, and won't retry any more
- FAILED_CLOSE = 10; // failed to close, and won't retry any more
- MERGING = 11; // server started merge a region
- MERGED = 12; // server completed merge of a region
- SPLITTING_NEW = 13; // new region to be created when RS splits a parent
- // region but hasn't be created yet, or master doesn't
- // know it's already created
- MERGING_NEW = 14; // new region to be created when RS merges two
- // daughter regions but hasn't be created yet, or
- // master doesn't know it's already created
- }
-}
-
-message RegionInTransition {
- required RegionSpecifier spec = 1;
- required RegionState region_state = 2;
-}
-
-/**
- * sequence Id of a store
- */
-message StoreSequenceId {
- required bytes family_name = 1;
- required uint64 sequence_id = 2;
-}
-
-/**
- * contains a sequence id of a region which should be the minimum of its store sequence ids and
- * list of sequence ids of the region's stores
- */
-message RegionStoreSequenceIds {
- required uint64 last_flushed_sequence_id = 1;
- repeated StoreSequenceId store_sequence_id = 2;
-}
-
-message RegionLoad {
- /** the region specifier */
- required RegionSpecifier region_specifier = 1;
-
- /** the number of stores for the region */
- optional uint32 stores = 2;
-
- /** the number of storefiles for the region */
- optional uint32 storefiles = 3;
-
- /** the total size of the store files for the region, uncompressed, in MB */
- optional uint32 store_uncompressed_size_MB = 4;
-
- /** the current total size of the store files for the region, in MB */
- optional uint32 storefile_size_MB = 5;
-
- /** the current size of the memstore for the region, in MB */
- optional uint32 memstore_size_MB = 6;
-
- /**
- * The current total size of root-level store file indexes for the region,
- * in KB. The same as {@link #rootIndexSizeKB}.
- */
- optional uint64 storefile_index_size_KB = 7;
-
- /** the current total read requests made to region */
- optional uint64 read_requests_count = 8;
-
- /** the current total write requests made to region */
- optional uint64 write_requests_count = 9;
-
- /** the total compacting key values in currently running compaction */
- optional uint64 total_compacting_KVs = 10;
-
- /** the completed count of key values in currently running compaction */
- optional uint64 current_compacted_KVs = 11;
-
- /** The current total size of root-level indexes for the region, in KB. */
- optional uint32 root_index_size_KB = 12;
-
- /** The total size of all index blocks, not just the root level, in KB. */
- optional uint32 total_static_index_size_KB = 13;
-
- /**
- * The total size of all Bloom filter blocks, not just loaded into the
- * block cache, in KB.
- */
- optional uint32 total_static_bloom_size_KB = 14;
-
- /** the most recent sequence Id from cache flush */
- optional uint64 complete_sequence_id = 15;
-
- /** The current data locality for region in the regionserver */
- optional float data_locality = 16;
-
- optional uint64 last_major_compaction_ts = 17 [default = 0];
-
- /** the most recent sequence Id of store from cache flush */
- repeated StoreSequenceId store_complete_sequence_id = 18;
-
- /** the current total filtered read requests made to region */
- optional uint64 filtered_read_requests_count = 19;
-
- /** the current total coprocessor requests made to region */
- optional uint64 cp_requests_count = 20;
-
- /** the number of references active on the store */
- optional int32 store_ref_count = 21 [default = 0];
-
- /**
- * The max number of references active on single store file among all compacted store files
- * that belong to given region
- */
- optional int32 max_compacted_store_file_ref_count = 22 [default = 0];
-}
-
-message UserLoad {
-
- /** short user name */
- required string userName = 1;
-
- /** Metrics for all clients of a user */
- repeated ClientMetrics clientMetrics = 2;
-
-
-}
-
-message ClientMetrics {
- /** client host name */
- required string hostName = 1;
-
- /** the current total read requests made from a client */
- optional uint64 read_requests_count = 2;
-
- /** the current total write requests made from a client */
- optional uint64 write_requests_count = 3;
-
- /** the current total filtered requests made from a client */
- optional uint64 filtered_requests_count = 4;
-
-}
-
-/* Server-level protobufs */
-
-message ReplicationLoadSink {
- required uint64 ageOfLastAppliedOp = 1;
- required uint64 timeStampsOfLastAppliedOp = 2;
-}
-
-message ReplicationLoadSource {
- required string peerID = 1;
- required uint64 ageOfLastShippedOp = 2;
- required uint32 sizeOfLogQueue = 3;
- required uint64 timeStampOfLastShippedOp = 4;
- required uint64 replicationLag = 5;
-}
-
-message ServerLoad {
- /** Number of requests since last report. */
- optional uint64 number_of_requests = 1;
-
- /** Total Number of requests from the start of the region server. */
- optional uint64 total_number_of_requests = 2;
-
- /** the amount of used heap, in MB. */
- optional uint32 used_heap_MB = 3;
-
- /** the maximum allowable size of the heap, in MB. */
- optional uint32 max_heap_MB = 4;
-
- /** Information on the load of individual regions. */
- repeated RegionLoad region_loads = 5;
-
- /**
- * Regionserver-level coprocessors, e.g., WALObserver implementations.
- * Region-level coprocessors, on the other hand, are stored inside RegionLoad
- * objects.
- */
- repeated Coprocessor coprocessors = 6;
-
- /**
- * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
- * time is measured as the difference, measured in milliseconds, between the current time
- * and midnight, January 1, 1970 UTC.
- */
- optional uint64 report_start_time = 7;
-
- /**
- * Time when report was generated.
- * time is measured as the difference, measured in milliseconds, between the current time
- * and midnight, January 1, 1970 UTC.
- */
- optional uint64 report_end_time = 8;
-
- /**
- * The port number that this region server is hosing an info server on.
- */
- optional uint32 info_server_port = 9;
-
- /**
- * The replicationLoadSource for the replication Source status of this region server.
- */
- repeated ReplicationLoadSource replLoadSource = 10;
-
- /**
- * The replicationLoadSink for the replication Sink status of this region server.
- */
- optional ReplicationLoadSink replLoadSink = 11;
-
- /**
- * The metrics for each user on this region server
- */
- repeated UserLoad userLoads = 12;
-}
-
-message LiveServerInfo {
- required ServerName server = 1;
- required ServerLoad server_load = 2;
-}
-
-message ClusterStatus {
- optional HBaseVersionFileContent hbase_version = 1;
- repeated LiveServerInfo live_servers = 2;
- repeated ServerName dead_servers = 3;
- repeated RegionInTransition regions_in_transition = 4;
- optional ClusterId cluster_id = 5;
- repeated Coprocessor master_coprocessors = 6;
- optional ServerName master = 7;
- repeated ServerName backup_masters = 8;
- optional bool balancer_on = 9;
-}
-
-enum Option {
- HBASE_VERSION = 0;
- CLUSTER_ID = 1;
- LIVE_SERVERS = 2;
- DEAD_SERVERS = 3;
- MASTER = 4;
- BACKUP_MASTERS = 5;
- MASTER_COPROCESSORS = 6;
- REGIONS_IN_TRANSITION = 7;
- BALANCER_ON = 8;
-}
diff --git a/hbase-protocol/src/main/protobuf/Comparator.proto b/hbase-protocol/src/main/protobuf/Comparator.proto
deleted file mode 100644
index 0a59cf3874b..00000000000
--- a/hbase-protocol/src/main/protobuf/Comparator.proto
+++ /dev/null
@@ -1,84 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers that are used for filters
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ComparatorProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-// This file contains protocol buffers that are used for comparators (e.g. in filters)
-
-message Comparator {
- required string name = 1;
- optional bytes serialized_comparator = 2;
-}
-
-message ByteArrayComparable {
- optional bytes value = 1;
-}
-
-message BinaryComparator {
- required ByteArrayComparable comparable = 1;
-}
-
-message LongComparator {
- required ByteArrayComparable comparable = 1;
-}
-
-message BinaryPrefixComparator {
- required ByteArrayComparable comparable = 1;
-}
-
-message BitComparator {
- required ByteArrayComparable comparable = 1;
- required BitwiseOp bitwise_op = 2;
-
- enum BitwiseOp {
- AND = 1;
- OR = 2;
- XOR = 3;
- }
-}
-
-message NullComparator {
-}
-
-message RegexStringComparator {
- required string pattern = 1;
- required int32 pattern_flags = 2;
- required string charset = 3;
- optional string engine = 4;
-}
-
-message SubstringComparator {
- required string substr = 1;
-}
-
-message BigDecimalComparator {
- required ByteArrayComparable comparable = 1;
-}
-
-message BinaryComponentComparator {
- required bytes value = 1;
- required uint32 offset = 2;
-}
diff --git a/hbase-protocol/src/main/protobuf/Encryption.proto b/hbase-protocol/src/main/protobuf/Encryption.proto
deleted file mode 100644
index e08ca2b0481..00000000000
--- a/hbase-protocol/src/main/protobuf/Encryption.proto
+++ /dev/null
@@ -1,34 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers used for encryption
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "EncryptionProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-message WrappedKey {
- required string algorithm = 1;
- required uint32 length = 2;
- required bytes data = 3;
- optional bytes iv = 4;
- optional bytes hash = 5;
-}
diff --git a/hbase-protocol/src/main/protobuf/ErrorHandling.proto b/hbase-protocol/src/main/protobuf/ErrorHandling.proto
deleted file mode 100644
index c57a6d25250..00000000000
--- a/hbase-protocol/src/main/protobuf/ErrorHandling.proto
+++ /dev/null
@@ -1,59 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers that are used for error handling
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ErrorHandlingProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-/**
- * Protobuf version of a java.lang.StackTraceElement
- * so we can serialize exceptions.
- */
-message StackTraceElementMessage {
- optional string declaring_class = 1;
- optional string method_name = 2;
- optional string file_name = 3;
- optional int32 line_number = 4;
-}
-
-/**
- * Cause of a remote failure for a generic exception. Contains
- * all the information for a generic exception as well as
- * optional info about the error for generic info passing
- * (which should be another protobuffed class).
- */
-message GenericExceptionMessage {
- optional string class_name = 1;
- optional string message = 2;
- optional bytes error_info = 3;
- repeated StackTraceElementMessage trace = 4;
-}
-
-/**
- * Exception sent across the wire when a remote task needs
- * to notify other tasks that it failed and why
- */
-message ForeignExceptionMessage {
- optional string source = 1;
- optional GenericExceptionMessage generic_exception = 2;
-}
diff --git a/hbase-protocol/src/main/protobuf/FS.proto b/hbase-protocol/src/main/protobuf/FS.proto
deleted file mode 100644
index 090617972fa..00000000000
--- a/hbase-protocol/src/main/protobuf/FS.proto
+++ /dev/null
@@ -1,46 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers that are written into the filesystem
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "FSProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-/**
- * The ${HBASE_ROOTDIR}/hbase.version file content
- */
-message HBaseVersionFileContent {
- required string version = 1;
-}
-
-/**
- * Reference file content used when we split an hfile under a region.
- */
-message Reference {
- required bytes splitkey = 1;
- enum Range {
- TOP = 0;
- BOTTOM = 1;
- }
- required Range range = 2;
-}
-
diff --git a/hbase-protocol/src/main/protobuf/Filter.proto b/hbase-protocol/src/main/protobuf/Filter.proto
deleted file mode 100644
index 0b4c2013f4c..00000000000
--- a/hbase-protocol/src/main/protobuf/Filter.proto
+++ /dev/null
@@ -1,179 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers that are used for filters
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "FilterProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "Comparator.proto";
-
-message Filter {
- required string name = 1;
- optional bytes serialized_filter = 2;
-}
-
-message ColumnCountGetFilter {
- required int32 limit = 1;
-}
-
-message ColumnPaginationFilter {
- required int32 limit = 1;
- optional int32 offset = 2;
- optional bytes column_offset = 3;
-}
-
-message ColumnPrefixFilter {
- required bytes prefix = 1;
-}
-
-message ColumnRangeFilter {
- optional bytes min_column = 1;
- optional bool min_column_inclusive = 2;
- optional bytes max_column = 3;
- optional bool max_column_inclusive = 4;
-}
-
-message CompareFilter {
- required CompareType compare_op = 1;
- optional Comparator comparator = 2;
-}
-
-message DependentColumnFilter {
- required CompareFilter compare_filter = 1;
- optional bytes column_family = 2;
- optional bytes column_qualifier = 3;
- optional bool drop_dependent_column = 4;
-}
-
-message FamilyFilter {
- required CompareFilter compare_filter = 1;
-}
-
-message FilterList {
- required Operator operator = 1;
- repeated Filter filters = 2;
-
- enum Operator {
- MUST_PASS_ALL = 1;
- MUST_PASS_ONE = 2;
- }
-}
-
-message FilterWrapper {
- required Filter filter = 1;
-}
-
-message FirstKeyOnlyFilter {
-}
-
-message FirstKeyValueMatchingQualifiersFilter {
- repeated bytes qualifiers = 1;
-}
-
-message FuzzyRowFilter {
- repeated BytesBytesPair fuzzy_keys_data = 1;
-}
-
-message InclusiveStopFilter {
- optional bytes stop_row_key = 1;
-}
-
-message KeyOnlyFilter {
- required bool len_as_val = 1;
-}
-
-message MultipleColumnPrefixFilter {
- repeated bytes sorted_prefixes = 1;
-}
-
-message PageFilter {
- required int64 page_size = 1;
-}
-
-message PrefixFilter {
- optional bytes prefix = 1;
-}
-
-message QualifierFilter {
- required CompareFilter compare_filter = 1;
-}
-
-message RandomRowFilter {
- required float chance = 1;
-}
-
-message RowFilter {
- required CompareFilter compare_filter = 1;
-}
-
-message SingleColumnValueExcludeFilter {
- required SingleColumnValueFilter single_column_value_filter = 1;
-}
-
-message SingleColumnValueFilter {
- optional bytes column_family = 1;
- optional bytes column_qualifier = 2;
- required CompareType compare_op = 3;
- required Comparator comparator = 4;
- optional bool filter_if_missing = 5;
- optional bool latest_version_only = 6;
-}
-
-message SkipFilter {
- required Filter filter = 1;
-}
-
-message TimestampsFilter {
- repeated int64 timestamps = 1 [packed=true];
- optional bool can_hint = 2;
-}
-
-message ValueFilter {
- required CompareFilter compare_filter = 1;
-}
-
-message WhileMatchFilter {
- required Filter filter = 1;
-}
-message FilterAllFilter {
-}
-
-message RowRange {
- optional bytes start_row = 1;
- optional bool start_row_inclusive = 2;
- optional bytes stop_row = 3;
- optional bool stop_row_inclusive =4;
-}
-
-message MultiRowRangeFilter {
- repeated RowRange row_range_list = 1;
-}
-
-message ColumnValueFilter {
- required bytes family = 1;
- required bytes qualifier = 2;
- required CompareType compare_op = 3;
- required Comparator comparator = 4;
-}
diff --git a/hbase-protocol/src/main/protobuf/HBase.proto b/hbase-protocol/src/main/protobuf/HBase.proto
deleted file mode 100644
index f324aae7bb0..00000000000
--- a/hbase-protocol/src/main/protobuf/HBase.proto
+++ /dev/null
@@ -1,254 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers that are shared throughout HBase
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "HBaseProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-
-/**
- * Table Name
- */
-message TableName {
- required bytes namespace = 1;
- required bytes qualifier = 2;
-}
-
-/**
- * Table Schema
- * Inspired by the rest TableSchema
- */
-message TableSchema {
- optional TableName table_name = 1;
- repeated BytesBytesPair attributes = 2;
- repeated ColumnFamilySchema column_families = 3;
- repeated NameStringPair configuration = 4;
-}
-
-/** Denotes state of the table */
-message TableState {
- // Table's current state
- enum State {
- ENABLED = 0;
- DISABLED = 1;
- DISABLING = 2;
- ENABLING = 3;
- }
- // This is the table's state.
- required State state = 1;
-}
-
-/**
- * Column Family Schema
- * Inspired by the rest ColumSchemaMessage
- */
-message ColumnFamilySchema {
- required bytes name = 1;
- repeated BytesBytesPair attributes = 2;
- repeated NameStringPair configuration = 3;
-}
-
-/**
- * Protocol buffer version of HRegionInfo.
- */
-message RegionInfo {
- required uint64 region_id = 1;
- required TableName table_name = 2;
- optional bytes start_key = 3;
- optional bytes end_key = 4;
- optional bool offline = 5;
- optional bool split = 6;
- optional int32 replica_id = 7 [default = 0];
-}
-
-/**
- * Protocol buffer for favored nodes
- */
-message FavoredNodes {
- repeated ServerName favored_node = 1;
-}
-
-/**
- * Container protocol buffer to specify a region.
- * You can specify region by region name, or the hash
- * of the region name, which is known as encoded
- * region name.
- */
-message RegionSpecifier {
- required RegionSpecifierType type = 1;
- required bytes value = 2;
-
- enum RegionSpecifierType {
- // ,,.
- REGION_NAME = 1;
-
- // hash of ,,
- ENCODED_REGION_NAME = 2;
- }
-}
-
-/**
- * A range of time. Both from and to are Java time
- * stamp in milliseconds. If you don't specify a time
- * range, it means all time. By default, if not
- * specified, from = 0, and to = Long.MAX_VALUE
- */
-message TimeRange {
- optional uint64 from = 1;
- optional uint64 to = 2;
-}
-
-/* ColumnFamily Specific TimeRange */
-message ColumnFamilyTimeRange {
- required bytes column_family = 1;
- required TimeRange time_range = 2;
-}
-
-/* Comparison operators */
-enum CompareType {
- LESS = 0;
- LESS_OR_EQUAL = 1;
- EQUAL = 2;
- NOT_EQUAL = 3;
- GREATER_OR_EQUAL = 4;
- GREATER = 5;
- NO_OP = 6;
-}
-
-/**
- * Protocol buffer version of ServerName
- */
-message ServerName {
- required string host_name = 1;
- optional uint32 port = 2;
- optional uint64 start_code = 3;
-}
-
-// Comment data structures
-
-message Coprocessor {
- required string name = 1;
-}
-
-message NameStringPair {
- required string name = 1;
- required string value = 2;
-}
-
-message NameBytesPair {
- required string name = 1;
- optional bytes value = 2;
-}
-
-message BytesBytesPair {
- required bytes first = 1;
- required bytes second = 2;
-}
-
-message NameInt64Pair {
- optional string name = 1;
- optional int64 value = 2;
-}
-
-/**
- * Description of the snapshot to take
- */
-message SnapshotDescription {
- required string name = 1;
- optional string table = 2; // not needed for delete, but checked for in taking snapshot
- optional int64 creation_time = 3 [default = 0];
- enum Type {
- DISABLED = 0;
- FLUSH = 1;
- SKIPFLUSH = 2;
- }
- optional Type type = 4 [default = FLUSH];
- optional int32 version = 5;
- optional string owner = 6;
- optional int64 ttl = 7 [default = 0];
-}
-
-/**
- * Description of the distributed procedure to take
- */
-message ProcedureDescription {
- required string signature = 1; // the unique signature of the procedure
- optional string instance = 2; // the procedure instance name
- optional int64 creation_time = 3 [default = 0];
- repeated NameStringPair configuration = 4;
-}
-
-message EmptyMsg {
-}
-
-enum TimeUnit {
- NANOSECONDS = 1;
- MICROSECONDS = 2;
- MILLISECONDS = 3;
- SECONDS = 4;
- MINUTES = 5;
- HOURS = 6;
- DAYS = 7;
-}
-
-message LongMsg {
- required int64 long_msg = 1;
-}
-
-message DoubleMsg {
- required double double_msg = 1;
-}
-
-message BigDecimalMsg {
- required bytes bigdecimal_msg = 1;
-}
-
-message UUID {
- required uint64 least_sig_bits = 1;
- required uint64 most_sig_bits = 2;
-}
-
-message NamespaceDescriptor {
- required bytes name = 1;
- repeated NameStringPair configuration = 2;
-}
-
-// Rpc client version info proto. Included in ConnectionHeader on connection setup
-message VersionInfo {
- required string version = 1;
- required string url = 2;
- required string revision = 3;
- required string user = 4;
- required string date = 5;
- required string src_checksum = 6;
- optional uint32 version_major = 7;
- optional uint32 version_minor = 8;
-}
-
-/**
- * Description of the region server info
- */
-message RegionServerInfo {
- optional int32 infoPort = 1;
- optional VersionInfo version_info = 2;
-}
diff --git a/hbase-protocol/src/main/protobuf/HFile.proto b/hbase-protocol/src/main/protobuf/HFile.proto
deleted file mode 100644
index df8f57ee238..00000000000
--- a/hbase-protocol/src/main/protobuf/HFile.proto
+++ /dev/null
@@ -1,50 +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.
- */
-syntax = "proto2";
-
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "HFileProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-// Map of name/values
-message FileInfoProto {
- repeated BytesBytesPair map_entry = 1;
-}
-
-// HFile file trailer
-message FileTrailerProto {
- optional uint64 file_info_offset = 1;
- optional uint64 load_on_open_data_offset = 2;
- optional uint64 uncompressed_data_index_size = 3;
- optional uint64 total_uncompressed_bytes = 4;
- optional uint32 data_index_count = 5;
- optional uint32 meta_index_count = 6;
- optional uint64 entry_count = 7;
- optional uint32 num_data_index_levels = 8;
- optional uint64 first_data_block_offset = 9;
- optional uint64 last_data_block_offset = 10;
- optional string comparator_class_name = 11;
- optional uint32 compression_codec = 12;
- optional bytes encryption_key = 13;
-}
diff --git a/hbase-protocol/src/main/protobuf/LoadBalancer.proto b/hbase-protocol/src/main/protobuf/LoadBalancer.proto
deleted file mode 100644
index 0c0882170c7..00000000000
--- a/hbase-protocol/src/main/protobuf/LoadBalancer.proto
+++ /dev/null
@@ -1,30 +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.
- */
-syntax = "proto2";
-
-// This file contains protocol buffers to represent the state of the load balancer.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "LoadBalancerProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-message LoadBalancerState {
- optional bool balancer_on = 1;
-}
diff --git a/hbase-protocol/src/main/protobuf/MapReduce.proto b/hbase-protocol/src/main/protobuf/MapReduce.proto
deleted file mode 100644
index 7fb6850539f..00000000000
--- a/hbase-protocol/src/main/protobuf/MapReduce.proto
+++ /dev/null
@@ -1,38 +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.
- */
-syntax = "proto2";
-
- //This file includes protocol buffers used in MapReduce only.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "MapReduceProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-message ScanMetrics {
- repeated NameInt64Pair metrics = 1;
-}
-
-message TableSnapshotRegionSplit {
- repeated string locations = 2;
- optional TableSchema table = 3;
- optional RegionInfo region = 4;
-}
diff --git a/hbase-protocol/src/main/protobuf/MultiRowMutation.proto b/hbase-protocol/src/main/protobuf/MultiRowMutation.proto
deleted file mode 100644
index d3140e9b2e0..00000000000
--- a/hbase-protocol/src/main/protobuf/MultiRowMutation.proto
+++ /dev/null
@@ -1,48 +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.
- */
-syntax = "proto2";
-package hbase.pb;
-
-import "Client.proto";
-import "HBase.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;
- optional RegionSpecifier region = 4;
-}
-
-message MutateRowsResponse {
-}
-
-service MultiRowMutationService {
- rpc MutateRows(MutateRowsRequest)
- returns(MutateRowsResponse);
-}
diff --git a/hbase-protocol/src/main/protobuf/PingProtocol.proto b/hbase-protocol/src/main/protobuf/PingProtocol.proto
deleted file mode 100644
index 9a645994886..00000000000
--- a/hbase-protocol/src/main/protobuf/PingProtocol.proto
+++ /dev/null
@@ -1,68 +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.
- */
-syntax = "proto2";
-
-// Coprocessor test
-option java_package = "org.apache.hadoop.hbase.coprocessor.protobuf.generated";
-option java_outer_classname = "PingProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-message PingRequest {
-}
-
-message PingResponse {
- required string pong = 1;
-}
-
-message CountRequest {
-}
-
-message CountResponse {
- required int32 count = 1;
-}
-
-message IncrementCountRequest {
- required int32 diff = 1;
-}
-
-message IncrementCountResponse {
- required int32 count = 1;
-}
-
-message HelloRequest {
- optional string name = 1;
-}
-
-message HelloResponse {
- optional string response = 1;
-}
-
-message NoopRequest {
-}
-
-message NoopResponse {
-}
-
-service PingService {
- rpc ping(PingRequest) returns(PingResponse);
- rpc count(CountRequest) returns(CountResponse);
- rpc increment(IncrementCountRequest) returns(IncrementCountResponse);
- rpc hello(HelloRequest) returns(HelloResponse);
- rpc noop(NoopRequest) returns(NoopResponse);
-}
diff --git a/hbase-protocol/src/main/protobuf/Quota.proto b/hbase-protocol/src/main/protobuf/Quota.proto
deleted file mode 100644
index fa5462c5f1b..00000000000
--- a/hbase-protocol/src/main/protobuf/Quota.proto
+++ /dev/null
@@ -1,113 +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.
- */
-syntax = "proto2";
-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;
- SPACE = 2;
-}
-
-message Quotas {
- optional bool bypass_globals = 1 [default = false];
- optional Throttle throttle = 2;
- optional SpaceQuota space = 3;
-}
-
-message QuotaUsage {
-}
-
-// Defines what action should be taken when the SpaceQuota is violated
-enum SpaceViolationPolicy {
- DISABLE = 1; // Disable the table(s)
- NO_WRITES_COMPACTIONS = 2; // No writes, bulk-loads, or compactions
- NO_WRITES = 3; // No writes or bulk-loads
- NO_INSERTS = 4; // No puts or bulk-loads, but deletes are allowed
-}
-
-// Defines a limit on the amount of filesystem space used by a table/namespace
-message SpaceQuota {
- optional uint64 soft_limit = 1; // The limit of bytes for this quota
- optional SpaceViolationPolicy violation_policy = 2; // The action to take when the quota is violated
- optional bool remove = 3 [default = false]; // When true, remove the quota.
-}
-
-// The Request to limit space usage (to allow for schema evolution not tied to SpaceQuota).
-message SpaceLimitRequest {
- optional SpaceQuota quota = 1;
-}
-
-// Represents the state of a quota on a table. Either the quota is not in violation
-// or it is in violatino there is a violation policy which should be in effect.
-message SpaceQuotaStatus {
- optional SpaceViolationPolicy violation_policy = 1;
- optional bool in_violation = 2;
-}
-
-// Message stored in the value of hbase:quota table to denote the status of a table WRT
-// the quota applicable to it.
-message SpaceQuotaSnapshot {
- optional SpaceQuotaStatus quota_status = 1;
- optional uint64 quota_usage = 2;
- optional uint64 quota_limit = 3;
-}
diff --git a/hbase-protocol/src/main/protobuf/RPC.proto b/hbase-protocol/src/main/protobuf/RPC.proto
deleted file mode 100644
index 25e051430e2..00000000000
--- a/hbase-protocol/src/main/protobuf/RPC.proto
+++ /dev/null
@@ -1,138 +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.
- */
-syntax = "proto2";
-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:
-//
-//
-//
-//
-//
-//
-// ...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:
-//
-//
-//
-//
-//
-//
-// ...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;
- optional uint32 timeout = 7;
-}
-
-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;
-}
diff --git a/hbase-protocol/src/main/protobuf/RSGroup.proto b/hbase-protocol/src/main/protobuf/RSGroup.proto
deleted file mode 100644
index 31a7716dfd7..00000000000
--- a/hbase-protocol/src/main/protobuf/RSGroup.proto
+++ /dev/null
@@ -1,34 +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.
- */
-syntax = "proto2";
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "RSGroupProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-message RSGroupInfo {
- required string name = 1;
- repeated ServerName servers = 4;
- repeated TableName tables = 3;
-}
-
diff --git a/hbase-protocol/src/main/protobuf/RSGroupAdmin.proto b/hbase-protocol/src/main/protobuf/RSGroupAdmin.proto
deleted file mode 100644
index b73e370fb2f..00000000000
--- a/hbase-protocol/src/main/protobuf/RSGroupAdmin.proto
+++ /dev/null
@@ -1,158 +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.
- */
-syntax = "proto2";
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "RSGroupAdminProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "RSGroup.proto";
-
-/** Group level protobufs */
-
-message ListTablesOfRSGroupRequest {
- required string r_s_group_name = 1;
-}
-
-message ListTablesOfRSGroupResponse {
- repeated TableName table_name = 1;
-}
-
-message GetRSGroupInfoRequest {
- required string r_s_group_name = 1;
-}
-
-message GetRSGroupInfoResponse {
- optional RSGroupInfo r_s_group_info = 1;
-}
-
-message GetRSGroupInfoOfTableRequest {
- required TableName table_name = 1;
-}
-
-message GetRSGroupInfoOfTableResponse {
- optional RSGroupInfo r_s_group_info = 1;
-}
-
-message MoveServersRequest {
- required string target_group = 1;
- repeated ServerName servers = 3;
-}
-
-message MoveServersResponse {
-}
-
-message MoveTablesRequest {
- required string target_group = 1;
- repeated TableName table_name = 2;
-}
-
-message MoveTablesResponse {
-}
-
-message AddRSGroupRequest {
- required string r_s_group_name = 1;
-}
-
-message AddRSGroupResponse {
-}
-
-message RemoveRSGroupRequest {
- required string r_s_group_name = 1;
-}
-
-message RemoveRSGroupResponse {
-}
-
-message BalanceRSGroupRequest {
- required string r_s_group_name = 1;
-}
-
-message BalanceRSGroupResponse {
- required bool balanceRan = 1;
-}
-
-message ListRSGroupInfosRequest {
-}
-
-message ListRSGroupInfosResponse {
- repeated RSGroupInfo r_s_group_info = 1;
-}
-
-message GetRSGroupInfoOfServerRequest {
- required ServerName server = 2;
-}
-
-message GetRSGroupInfoOfServerResponse {
- optional RSGroupInfo r_s_group_info = 1;
-}
-
-message MoveServersAndTablesRequest {
- required string target_group = 1;
- repeated ServerName servers = 2;
- repeated TableName table_name = 3;
-}
-
-message MoveServersAndTablesResponse {
-}
-
-message RemoveServersRequest {
- repeated ServerName servers = 1;
-}
-
-message RemoveServersResponse {
-}
-
-service RSGroupAdminService {
- rpc GetRSGroupInfo(GetRSGroupInfoRequest)
- returns (GetRSGroupInfoResponse);
-
- rpc GetRSGroupInfoOfTable(GetRSGroupInfoOfTableRequest)
- returns (GetRSGroupInfoOfTableResponse);
-
- rpc GetRSGroupInfoOfServer(GetRSGroupInfoOfServerRequest)
- returns (GetRSGroupInfoOfServerResponse);
-
- rpc MoveServers(MoveServersRequest)
- returns (MoveServersResponse);
-
- rpc MoveTables(MoveTablesRequest)
- returns (MoveTablesResponse);
-
- rpc AddRSGroup(AddRSGroupRequest)
- returns (AddRSGroupResponse);
-
- rpc RemoveRSGroup(RemoveRSGroupRequest)
- returns (RemoveRSGroupResponse);
-
- rpc BalanceRSGroup(BalanceRSGroupRequest)
- returns (BalanceRSGroupResponse);
-
- rpc ListRSGroupInfos(ListRSGroupInfosRequest)
- returns (ListRSGroupInfosResponse);
-
- rpc MoveServersAndTables(MoveServersAndTablesRequest)
- returns (MoveServersAndTablesResponse);
-
- rpc RemoveServers(RemoveServersRequest)
- returns (RemoveServersResponse);
-}
diff --git a/hbase-protocol/src/main/protobuf/RowProcessor.proto b/hbase-protocol/src/main/protobuf/RowProcessor.proto
deleted file mode 100644
index b2ed362df74..00000000000
--- a/hbase-protocol/src/main/protobuf/RowProcessor.proto
+++ /dev/null
@@ -1,47 +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.
- */
-syntax = "proto2";
-
-/**
- * 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);
-}
diff --git a/hbase-protocol/src/main/protobuf/Snapshot.proto b/hbase-protocol/src/main/protobuf/Snapshot.proto
deleted file mode 100644
index 59a65a9a662..00000000000
--- a/hbase-protocol/src/main/protobuf/Snapshot.proto
+++ /dev/null
@@ -1,67 +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.
- */
-syntax = "proto2";
-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;
-}
diff --git a/hbase-protocol/src/main/protobuf/Tracing.proto b/hbase-protocol/src/main/protobuf/Tracing.proto
deleted file mode 100644
index 4fa1ab8c12d..00000000000
--- a/hbase-protocol/src/main/protobuf/Tracing.proto
+++ /dev/null
@@ -1,34 +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.
- */
-syntax = "proto2";
-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;
-}
diff --git a/hbase-protocol/src/main/protobuf/VisibilityLabels.proto b/hbase-protocol/src/main/protobuf/VisibilityLabels.proto
deleted file mode 100644
index 44d0dfc4944..00000000000
--- a/hbase-protocol/src/main/protobuf/VisibilityLabels.proto
+++ /dev/null
@@ -1,84 +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.
- */
-syntax = "proto2";
-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);
-}
diff --git a/hbase-protocol/src/main/protobuf/WAL.proto b/hbase-protocol/src/main/protobuf/WAL.proto
deleted file mode 100644
index f24ea0fc505..00000000000
--- a/hbase-protocol/src/main/protobuf/WAL.proto
+++ /dev/null
@@ -1,177 +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.
- */
-syntax = "proto2";
-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";
-
-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;
- repeated Attribute extended_attributes = 12;
-/*
- optional CustomEntryType custom_entry_type = 9;
-
- enum CustomEntryType {
- COMPACTION = 0;
- }
-*/
-}
-message Attribute {
- required string key = 1;
- required bytes value = 2;
-}
-
-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
- optional uint64 store_file_size_bytes = 4; // size of store file
-}
-
-/**
- * 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 {
-}
diff --git a/hbase-protocol/src/main/protobuf/ZooKeeper.proto b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
deleted file mode 100644
index e06f4a03a00..00000000000
--- a/hbase-protocol/src/main/protobuf/ZooKeeper.proto
+++ /dev/null
@@ -1,147 +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.
- */
-syntax = "proto2";
-
-// 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;
- }
- required State state = 1;
- required ServerName server_name = 2;
- // optional RecoveryMode DEPRECATED_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];
-}
-
-message TableCF {
- optional TableName table_name = 1;
- repeated bytes families = 2;
-}
-
-/**
- * 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;
- repeated TableCF table_cfs = 5;
- repeated bytes namespaces = 6;
- optional int64 bandwidth = 7;
-}
-
-/**
- * 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;
-}
-
-/**
- * State of the switch.
- */
-message SwitchState {
- optional bool enabled = 1;
-}
diff --git a/hbase-protocol/src/main/protobuf/test.proto b/hbase-protocol/src/main/protobuf/test.proto
deleted file mode 100644
index 89d47865c39..00000000000
--- a/hbase-protocol/src/main/protobuf/test.proto
+++ /dev/null
@@ -1,44 +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.
- */
-syntax = "proto2";
-
-option java_package = "org.apache.hadoop.hbase.ipc.protobuf.generated";
-option java_outer_classname = "TestProtos";
-option java_generate_equals_and_hash = true;
-
-message EmptyRequestProto {
-}
-
-message EmptyResponseProto {
-}
-
-message EchoRequestProto {
- required string message = 1;
-}
-
-message EchoResponseProto {
- required string message = 1;
-}
-
-message PauseRequestProto {
- required uint32 ms = 1;
-}
-
-message AddrResponseProto {
- required string addr = 1;
-}
diff --git a/hbase-protocol/src/main/protobuf/test_rpc_service.proto b/hbase-protocol/src/main/protobuf/test_rpc_service.proto
deleted file mode 100644
index 7c67ef2dd43..00000000000
--- a/hbase-protocol/src/main/protobuf/test_rpc_service.proto
+++ /dev/null
@@ -1,37 +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.
- */
-syntax = "proto2";
-
-option java_package = "org.apache.hadoop.hbase.ipc.protobuf.generated";
-option java_outer_classname = "TestRpcServiceProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-import "test.proto";
-
-
-/**
- * A protobuf service for use in tests
- */
-service TestProtobufRpcProto {
- rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
- rpc echo(EchoRequestProto) returns (EchoResponseProto);
- rpc error(EmptyRequestProto) returns (EmptyResponseProto);
- rpc pause(PauseRequestProto) returns (EmptyResponseProto);
- rpc addr(EmptyRequestProto) returns (AddrResponseProto);
-}
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/CellModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/CellModel.java
index 2621de0eacb..128be02bb34 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/CellModel.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/CellModel.java
@@ -19,28 +19,26 @@
package org.apache.hadoop.hbase.rest.model;
+import com.fasterxml.jackson.annotation.JsonProperty;
import java.io.IOException;
import java.io.Serializable;
-
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlAttribute;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlValue;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.CellMessage.Cell;
-
-import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
/**
* Representation of a cell. A cell is a single value associated a column and
* optional qualifier, and either the timestamp when it was stored or the user-
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 05813f5b067..5cf303ec0c3 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -266,13 +266,6 @@
test-jar
test
-
-
- org.apache.hbase
- hbase-protocol
-
org.apache.hbase
hbase-protocol-shaded
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index b5ca6b5af67..eba262f64a0 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -50,7 +50,6 @@ org.apache.hadoop.hbase.master.DeadServer;
org.apache.hadoop.hbase.master.HMaster;
org.apache.hadoop.hbase.master.RegionState;
org.apache.hadoop.hbase.master.ServerManager;
-org.apache.hadoop.hbase.protobuf.ProtobufUtil;
org.apache.hadoop.hbase.quotas.QuotaUtil;
org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;
org.apache.hadoop.hbase.rsgroup.RSGroupUtil;
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
index bf94c5b29b5..7947f294cd1 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
@@ -31,9 +31,6 @@ org.apache.hadoop.hbase.util.Bytes;
org.apache.hadoop.hbase.HRegionInfo;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.HBaseConfiguration;
-org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
-org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
org.apache.hadoop.hbase.util.DirectMemoryUtils;
org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
java.lang.management.MemoryUsage;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
index 086f85b26ec..842b25db513 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
@@ -358,12 +358,11 @@ public final class SnapshotDescriptionUtils {
}
/**
- * Read in the {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription} stored for the snapshot in the passed directory
+ * Read in the {@link SnapshotDescription} stored for the snapshot in the passed directory
* @param fs filesystem where the snapshot was taken
* @param snapshotDir directory where the snapshot was stored
* @return the stored snapshot description
- * @throws CorruptedSnapshotException if the
- * snapshot cannot be read
+ * @throws CorruptedSnapshotException if the snapshot cannot be read
*/
public static SnapshotDescription readSnapshotInfo(FileSystem fs, Path snapshotDir)
throws CorruptedSnapshotException {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
deleted file mode 100644
index 69e656fbe52..00000000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
+++ /dev/null
@@ -1,383 +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.
- */
-package org.apache.hadoop.hbase.protobuf;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import com.google.protobuf.ByteString;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellBuilderType;
-import org.apache.hadoop.hbase.CellComparatorImpl;
-import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.io.TimeRange;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
-
-/**
- * Class to test ProtobufUtil.
- */
-@Category({ MiscTests.class, SmallTests.class})
-public class TestProtobufUtil {
-
- @ClassRule
- public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestProtobufUtil.class);
-
- @Test
- public void testException() throws IOException {
- NameBytesPair.Builder builder = NameBytesPair.newBuilder();
- final String omg = "OMG!!!";
- builder.setName("java.io.IOException");
- builder.setValue(ByteString.copyFrom(Bytes.toBytes(omg)));
- Throwable t = ProtobufUtil.toException(builder.build());
- assertEquals(omg, t.getMessage());
- builder.clear();
- builder.setName("org.apache.hadoop.ipc.RemoteException");
- builder.setValue(ByteString.copyFrom(Bytes.toBytes(omg)));
- t = ProtobufUtil.toException(builder.build());
- assertEquals(omg, t.getMessage());
- }
-
- /**
- * Test basic Get conversions.
- *
- * @throws IOException
- */
- @Test
- public void testGet() throws IOException {
- ClientProtos.Get.Builder getBuilder = ClientProtos.Get.newBuilder();
- getBuilder.setRow(ByteString.copyFromUtf8("row"));
- Column.Builder columnBuilder = Column.newBuilder();
- columnBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- columnBuilder.addQualifier(ByteString.copyFromUtf8("c1"));
- columnBuilder.addQualifier(ByteString.copyFromUtf8("c2"));
- getBuilder.addColumn(columnBuilder.build());
-
- columnBuilder.clear();
- columnBuilder.setFamily(ByteString.copyFromUtf8("f2"));
- getBuilder.addColumn(columnBuilder.build());
- getBuilder.setLoadColumnFamiliesOnDemand(true);
- ClientProtos.Get proto = getBuilder.build();
- // default fields
- assertEquals(1, proto.getMaxVersions());
- assertEquals(true, proto.getCacheBlocks());
-
- // set the default value for equal comparison
- getBuilder = ClientProtos.Get.newBuilder(proto);
- getBuilder.setMaxVersions(1);
- getBuilder.setCacheBlocks(true);
- getBuilder.setTimeRange(ProtobufUtil.toTimeRange(TimeRange.allTime()));
-
- Get get = ProtobufUtil.toGet(proto);
- assertEquals(getBuilder.build(), ProtobufUtil.toGet(get));
- }
-
- /**
- * Test Append Mutate conversions.
- *
- * @throws IOException
- */
- @Test
- public void testAppend() throws IOException {
- long timeStamp = 111111;
- MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
- mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
- mutateBuilder.setMutateType(MutationType.APPEND);
- mutateBuilder.setTimestamp(timeStamp);
- ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
- valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
- qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
- qualifierBuilder.setTimestamp(timeStamp);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
- qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- mutateBuilder.addColumnValue(valueBuilder.build());
-
- MutationProto proto = mutateBuilder.build();
- // default fields
- assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
-
- // set the default value for equal comparison
- mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
-
- Append append = ProtobufUtil.toAppend(proto, null);
-
- // append always use the latest timestamp,
- // reset the timestamp to the original mutate
- mutateBuilder.setTimestamp(append.getTimestamp());
- mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(append.getTimeRange()));
- assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
- }
-
- /**
- * Test Delete Mutate conversions.
- *
- * @throws IOException
- */
- @Test
- public void testDelete() throws IOException {
- MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
- mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
- mutateBuilder.setMutateType(MutationType.DELETE);
- mutateBuilder.setTimestamp(111111);
- ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
- valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
- qualifierBuilder.setDeleteType(DeleteType.DELETE_ONE_VERSION);
- qualifierBuilder.setTimestamp(111222);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
- qualifierBuilder.setDeleteType(DeleteType.DELETE_MULTIPLE_VERSIONS);
- qualifierBuilder.setTimestamp(111333);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- mutateBuilder.addColumnValue(valueBuilder.build());
-
- MutationProto proto = mutateBuilder.build();
- // default fields
- assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
-
- // set the default value for equal comparison
- mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
-
- Delete delete = ProtobufUtil.toDelete(proto);
-
- // delete always have empty value,
- // add empty value to the original mutate
- for (ColumnValue.Builder column:
- mutateBuilder.getColumnValueBuilderList()) {
- for (QualifierValue.Builder qualifier:
- column.getQualifierValueBuilderList()) {
- qualifier.setValue(ByteString.EMPTY);
- }
- }
- assertEquals(mutateBuilder.build(),
- ProtobufUtil.toMutation(MutationType.DELETE, delete));
- }
-
- /**
- * Test Increment Mutate conversions.
- *
- * @throws IOException
- */
- @Test
- public void testIncrement() throws IOException {
- long timeStamp = 111111;
- MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
- mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
- mutateBuilder.setMutateType(MutationType.INCREMENT);
- ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
- valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
- qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(11L)));
- qualifierBuilder.setTimestamp(timeStamp);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
- qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(22L)));
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- mutateBuilder.addColumnValue(valueBuilder.build());
-
- MutationProto proto = mutateBuilder.build();
- // default fields
- assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
-
- // set the default value for equal comparison
- mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
-
- Increment increment = ProtobufUtil.toIncrement(proto, null);
- mutateBuilder.setTimestamp(increment.getTimestamp());
- mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(increment.getTimeRange()));
- assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.INCREMENT, increment));
- }
-
- /**
- * Test Put Mutate conversions.
- *
- * @throws IOException
- */
- @Test
- public void testPut() throws IOException {
- MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
- mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
- mutateBuilder.setMutateType(MutationType.PUT);
- mutateBuilder.setTimestamp(111111);
- ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
- valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
- qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
- qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
- qualifierBuilder.setTimestamp(222222);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- mutateBuilder.addColumnValue(valueBuilder.build());
-
- MutationProto proto = mutateBuilder.build();
- // default fields
- assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
-
- // set the default value for equal comparison
- mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
-
- Put put = ProtobufUtil.toPut(proto);
-
- // put value always use the default timestamp if no
- // value level timestamp specified,
- // add the timestamp to the original mutate
- long timestamp = put.getTimestamp();
- for (ColumnValue.Builder column:
- mutateBuilder.getColumnValueBuilderList()) {
- for (QualifierValue.Builder qualifier:
- column.getQualifierValueBuilderList()) {
- if (!qualifier.hasTimestamp()) {
- qualifier.setTimestamp(timestamp);
- }
- }
- }
- assertEquals(mutateBuilder.build(),
- ProtobufUtil.toMutation(MutationType.PUT, put));
- }
-
- /**
- * Test basic Scan conversions.
- *
- * @throws IOException
- */
- @Test
- public void testScan() throws IOException {
- ClientProtos.Scan.Builder scanBuilder = ClientProtos.Scan.newBuilder();
- scanBuilder.setStartRow(ByteString.copyFromUtf8("row1"));
- scanBuilder.setStopRow(ByteString.copyFromUtf8("row2"));
- Column.Builder columnBuilder = Column.newBuilder();
- columnBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- columnBuilder.addQualifier(ByteString.copyFromUtf8("c1"));
- columnBuilder.addQualifier(ByteString.copyFromUtf8("c2"));
- scanBuilder.addColumn(columnBuilder.build());
-
- columnBuilder.clear();
- columnBuilder.setFamily(ByteString.copyFromUtf8("f2"));
- scanBuilder.addColumn(columnBuilder.build());
-
- ClientProtos.Scan proto = scanBuilder.build();
-
- // Verify default values
- assertEquals(1, proto.getMaxVersions());
- assertEquals(true, proto.getCacheBlocks());
-
- // Verify fields survive ClientProtos.Scan -> Scan -> ClientProtos.Scan
- // conversion
- scanBuilder = ClientProtos.Scan.newBuilder(proto);
- scanBuilder.setMaxVersions(2);
- scanBuilder.setCacheBlocks(false);
- scanBuilder.setCaching(1024);
- scanBuilder.setTimeRange(ProtobufUtil.toTimeRange(TimeRange.allTime()));
- scanBuilder.setIncludeStopRow(false);
- ClientProtos.Scan expectedProto = scanBuilder.build();
-
- ClientProtos.Scan actualProto = ProtobufUtil.toScan(
- ProtobufUtil.toScan(expectedProto));
- assertEquals(expectedProto, actualProto);
- }
-
- @Test
- public void testToCell() throws Exception {
- KeyValue kv1 =
- new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]);
- KeyValue kv2 =
- new KeyValue(Bytes.toBytes("bbb"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]);
- KeyValue kv3 =
- new KeyValue(Bytes.toBytes("ccc"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]);
- byte[] arr = new byte[kv1.getLength() + kv2.getLength() + kv3.getLength()];
- System.arraycopy(kv1.getBuffer(), kv1.getOffset(), arr, 0, kv1.getLength());
- System.arraycopy(kv2.getBuffer(), kv2.getOffset(), arr, kv1.getLength(), kv2.getLength());
- System.arraycopy(kv3.getBuffer(), kv3.getOffset(), arr, kv1.getLength() + kv2.getLength(),
- kv3.getLength());
- ByteBuffer dbb = ByteBuffer.allocateDirect(arr.length);
- dbb.put(arr);
- ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(dbb, kv1.getLength(), kv2.getLength());
- CellProtos.Cell cell = ProtobufUtil.toCell(offheapKV);
- Cell newOffheapKV =
- ProtobufUtil.toCell(ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
- assertTrue(CellComparatorImpl.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
- }
-
- @Test
- public void testMetaRegionState() throws Exception {
- ServerName serverName = ServerName.valueOf("localhost", 1234, 5678);
- // New region state style.
- for (RegionState.State state: RegionState.State.values()) {
- RegionState regionState =
- new RegionState(RegionInfoBuilder.FIRST_META_REGIONINFO, state, serverName);
- MetaRegionServer metars = MetaRegionServer.newBuilder()
- .setServer(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(serverName))
- .setRpcVersion(HConstants.RPC_CURRENT_VERSION)
- .setState(state.convert()).build();
- // Serialize
- byte[] data = ProtobufUtil.prependPBMagic(metars.toByteArray());
- ProtobufUtil.prependPBMagic(data);
- // Deserialize
- RegionState regionStateNew =
- org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(data, 1);
- assertEquals(regionState.getServerName(), regionStateNew.getServerName());
- assertEquals(regionState.getState(), regionStateNew.getState());
- }
- // old style.
- RegionState rs =
- org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(
- serverName.getVersionedBytes(), 1);
- assertEquals(serverName, rs.getServerName());
- assertEquals(rs.getState(), RegionState.State.OPEN);
- }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java
index 236c03545be..7b0e6cbdd8f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java
@@ -17,7 +17,8 @@
*/
package org.apache.hadoop.hbase.protobuf;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java
index 7cdf2f36ec0..ca1fe8bc2b3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.SecurityTests;
@@ -48,6 +47,8 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
+
@Category({SecurityTests.class, MediumTests.class})
public class TestVisibilityLabelsWithSLGStack {
diff --git a/hbase-testing-util/pom.xml b/hbase-testing-util/pom.xml
index 69d6d71bd5f..05121871d92 100644
--- a/hbase-testing-util/pom.xml
+++ b/hbase-testing-util/pom.xml
@@ -55,12 +55,6 @@
-
- org.apache.hbase
- hbase-protocol
- jar
- compile
-
org.apache.hbase
hbase-client
diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml
index 943a7a29494..948a609a32f 100644
--- a/hbase-thrift/pom.xml
+++ b/hbase-thrift/pom.xml
@@ -165,10 +165,6 @@
test-jar
test
-
- org.apache.hbase
- hbase-protocol
-
org.apache.hbase
hbase-client
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java
index 9f92cdfde93..18fd3012a07 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.security.visibility.ScanLabelGenerator;
@@ -73,6 +72,8 @@ import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
+
@Category({ClientTests.class, MediumTests.class})
public class TestThriftHBaseServiceHandlerWithLabels {
diff --git a/pom.xml b/pom.xml
index dfde14405dc..68be13ee773 100755
--- a/pom.xml
+++ b/pom.xml
@@ -70,7 +70,6 @@
hbase-thrift
hbase-shell
hbase-protocol-shaded
- hbase-protocol
hbase-client
hbase-hadoop-compat
hbase-common
@@ -1682,11 +1681,6 @@
hbase-protocol-shaded
${project.version}
-
- org.apache.hbase
- hbase-protocol
- ${project.version}
-
org.apache.hbase
hbase-procedure