Skip to content

Commit

Permalink
HBASE-24056 Improve the layout of our proto files in hbase-protocol-s…
Browse files Browse the repository at this point in the history
…haded module (#1377)

proto files layout:
protobuf/client - client to server messages, client rpc service and protos, used in hbase-client exclusively;
protobuf/rest - hbase-rest messages;
protobuf/rpc - rpc and post-rpc tracing messages;
protobuf/server/coprocessor - coprocessor rpc services;
protobuf/server/coprocessor/example - coprocessors rpc services examples from hbase-examples;
protobuf/server/io - filesystem and hbase-server/io protos;
protobuf/server/maser - master rpc services and messages;
protobuf/server/region - region rpc services and messages (except client rpc service, which is in Client.proto);
protobuf/server/rsgroup - rsgroup protos;
protobuf/server/zookeeper - protos for zookeeper and ones used exclusively in hbase-zookeeper module;
protobuf/server - protos used across other server protos;
protobuf/test - protos used in tests;
protobuf/ - protos used across other protos, exclusive for hbase-mapreduce and hbase-backup, other protos.

Signed-off-by: Jan Hentschel <[email protected]>
Signed-off-by: Duo Zhang <[email protected]>
  • Loading branch information
semensanyok authored Apr 6, 2020
1 parent da67491 commit 740e937
Show file tree
Hide file tree
Showing 62 changed files with 113 additions and 96 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,6 @@
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;

import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos;
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.CountRequest;
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.CountResponse;
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.HelloRequest;
Expand All @@ -67,6 +66,7 @@
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.NoopResponse;
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.PingRequest;
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.PingResponse;
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.PingService;

@Category({RegionServerTests.class, MediumTests.class})
public class TestServerCustomProtocol {
Expand All @@ -80,7 +80,7 @@ public class TestServerCustomProtocol {
static final String HELLO = "Hello, ";

/* Test protocol implementation */
public static class PingHandler extends PingProtos.PingService implements RegionCoprocessor {
public static class PingHandler extends PingService implements RegionCoprocessor {
private int counter = 0;

@Override
Expand Down Expand Up @@ -202,14 +202,14 @@ public void testSingleProxy() throws Throwable {
LOG.info("Who are you");
hello(table, NOBODY, null);
LOG.info(NOBODY);
Map<byte [], Integer> intResults = table.coprocessorService(PingProtos.PingService.class,
Map<byte [], Integer> intResults = table.coprocessorService(PingService.class,
null, null,
new Batch.Call<PingProtos.PingService, Integer>() {
new Batch.Call<PingService, Integer>() {
@Override
public Integer call(PingProtos.PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<PingProtos.CountResponse> rpcCallback =
public Integer call(PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<CountResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
instance.count(null, PingProtos.CountRequest.newBuilder().build(), rpcCallback);
instance.count(null, CountRequest.newBuilder().build(), rpcCallback);
return rpcCallback.get().getCount();
}
});
Expand All @@ -219,15 +219,15 @@ public Integer call(PingProtos.PingService instance) throws IOException {
count = e.getValue();
}
final int diff = 5;
intResults = table.coprocessorService(PingProtos.PingService.class,
intResults = table.coprocessorService(PingService.class,
null, null,
new Batch.Call<PingProtos.PingService, Integer>() {
new Batch.Call<PingService, Integer>() {
@Override
public Integer call(PingProtos.PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<PingProtos.IncrementCountResponse> rpcCallback =
public Integer call(PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<IncrementCountResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
instance.increment(null,
PingProtos.IncrementCountRequest.newBuilder().setDiff(diff).build(),
IncrementCountRequest.newBuilder().setDiff(diff).build(),
rpcCallback);
return rpcCallback.get().getCount();
}
Expand Down Expand Up @@ -256,52 +256,52 @@ public Integer call(PingProtos.PingService instance) throws IOException {

private Map<byte [], String> hello(final Table table, final String send, final byte [] start,
final byte [] end) throws ServiceException, Throwable {
return table.coprocessorService(PingProtos.PingService.class,
return table.coprocessorService(PingService.class,
start, end,
new Batch.Call<PingProtos.PingService, String>() {
new Batch.Call<PingService, String>() {
@Override
public String call(PingProtos.PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<PingProtos.HelloResponse> rpcCallback =
public String call(PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<HelloResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
PingProtos.HelloRequest.Builder builder = PingProtos.HelloRequest.newBuilder();
HelloRequest.Builder builder = HelloRequest.newBuilder();
if (send != null) {
builder.setName(send);
}
instance.hello(null, builder.build(), rpcCallback);
PingProtos.HelloResponse r = rpcCallback.get();
HelloResponse r = rpcCallback.get();
return r != null && r.hasResponse()? r.getResponse(): null;
}
});
}

private Map<byte [], String> compoundOfHelloAndPing(final Table table, final byte [] start,
final byte [] end) throws ServiceException, Throwable {
return table.coprocessorService(PingProtos.PingService.class,
return table.coprocessorService(PingService.class,
start, end,
new Batch.Call<PingProtos.PingService, String>() {
new Batch.Call<PingService, String>() {
@Override
public String call(PingProtos.PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<PingProtos.HelloResponse> rpcCallback =
public String call(PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<HelloResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
PingProtos.HelloRequest.Builder builder = PingProtos.HelloRequest.newBuilder();
HelloRequest.Builder builder = HelloRequest.newBuilder();
// Call ping on same instance. Use result calling hello on same instance.
builder.setName(doPing(instance));
instance.hello(null, builder.build(), rpcCallback);
PingProtos.HelloResponse r = rpcCallback.get();
HelloResponse r = rpcCallback.get();
return r != null && r.hasResponse()? r.getResponse(): null;
}
});
}

private Map<byte [], String> noop(final Table table, final byte [] start, final byte [] end)
throws ServiceException, Throwable {
return table.coprocessorService(PingProtos.PingService.class, start, end,
new Batch.Call<PingProtos.PingService, String>() {
return table.coprocessorService(PingService.class, start, end,
new Batch.Call<PingService, String>() {
@Override
public String call(PingProtos.PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<PingProtos.NoopResponse> rpcCallback =
public String call(PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<NoopResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
PingProtos.NoopRequest.Builder builder = PingProtos.NoopRequest.newBuilder();
NoopRequest.Builder builder = NoopRequest.newBuilder();
instance.noop(null, builder.build(), rpcCallback);
rpcCallback.get();
// Looks like null is expected when void. That is what the test below is looking for
Expand All @@ -314,14 +314,14 @@ public String call(PingProtos.PingService instance) throws IOException {
public void testSingleMethod() throws Throwable {
try (Table table = util.getConnection().getTable(TEST_TABLE);
RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
Map<byte [], String> results = table.coprocessorService(PingProtos.PingService.class,
Map<byte [], String> results = table.coprocessorService(PingService.class,
null, ROW_A,
new Batch.Call<PingProtos.PingService, String>() {
new Batch.Call<PingService, String>() {
@Override
public String call(PingProtos.PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<PingProtos.PingResponse> rpcCallback =
public String call(PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<PingResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
instance.ping(null, PingProtos.PingRequest.newBuilder().build(), rpcCallback);
instance.ping(null, PingRequest.newBuilder().build(), rpcCallback);
return rpcCallback.get().getPong();
}
});
Expand Down Expand Up @@ -404,19 +404,19 @@ public void testRowRange() throws Throwable {

private Map<byte [], String> ping(final Table table, final byte [] start, final byte [] end)
throws ServiceException, Throwable {
return table.coprocessorService(PingProtos.PingService.class, start, end,
new Batch.Call<PingProtos.PingService, String>() {
return table.coprocessorService(PingService.class, start, end,
new Batch.Call<PingService, String>() {
@Override
public String call(PingProtos.PingService instance) throws IOException {
public String call(PingService instance) throws IOException {
return doPing(instance);
}
});
}

private static String doPing(PingProtos.PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<PingProtos.PingResponse> rpcCallback =
private static String doPing(PingService instance) throws IOException {
CoprocessorRpcUtils.BlockingRpcCallback<PingResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
instance.ping(null, PingProtos.PingRequest.newBuilder().build(), rpcCallback);
instance.ping(null, PingRequest.newBuilder().build(), rpcCallback);
return rpcCallback.get().getPong();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,9 @@
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
import org.apache.hbase.thirdparty.com.google.protobuf.Service;

import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.ExampleProtos;
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.RowCountProtos.CountRequest;
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.RowCountProtos.CountResponse;
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.RowCountProtos.RowCountService;

/**
* Sample coprocessor endpoint exposing a Service interface for counting rows and key values.
Expand All @@ -50,7 +52,7 @@
* </p>
*/
@InterfaceAudience.Private
public class RowCountEndpoint extends ExampleProtos.RowCountService implements RegionCoprocessor {
public class RowCountEndpoint extends RowCountService implements RegionCoprocessor {
private RegionCoprocessorEnvironment env;

public RowCountEndpoint() {
Expand All @@ -68,11 +70,11 @@ public Iterable<Service> getServices() {
* Returns a count of the rows in the region where this coprocessor is loaded.
*/
@Override
public void getRowCount(RpcController controller, ExampleProtos.CountRequest request,
RpcCallback<ExampleProtos.CountResponse> done) {
public void getRowCount(RpcController controller, CountRequest request,
RpcCallback<CountResponse> done) {
Scan scan = new Scan();
scan.setFilter(new FirstKeyOnlyFilter());
ExampleProtos.CountResponse response = null;
CountResponse response = null;
InternalScanner scanner = null;
try {
scanner = env.getRegion().getScanner(scan);
Expand All @@ -92,7 +94,7 @@ public void getRowCount(RpcController controller, ExampleProtos.CountRequest req
results.clear();
} while (hasMore);

response = ExampleProtos.CountResponse.newBuilder()
response = CountResponse.newBuilder()
.setCount(count).build();
} catch (IOException ioe) {
CoprocessorRpcUtils.setControllerException(controller, ioe);
Expand All @@ -110,9 +112,9 @@ public void getRowCount(RpcController controller, ExampleProtos.CountRequest req
* Returns a count of all KeyValues in the region where this coprocessor is loaded.
*/
@Override
public void getKeyValueCount(RpcController controller, ExampleProtos.CountRequest request,
RpcCallback<ExampleProtos.CountResponse> done) {
ExampleProtos.CountResponse response = null;
public void getKeyValueCount(RpcController controller, CountRequest request,
RpcCallback<CountResponse> done) {
CountResponse response = null;
InternalScanner scanner = null;
try {
scanner = env.getRegion().getScanner(new Scan());
Expand All @@ -127,7 +129,7 @@ public void getKeyValueCount(RpcController controller, ExampleProtos.CountReques
results.clear();
} while (hasMore);

response = ExampleProtos.CountResponse.newBuilder()
response = CountResponse.newBuilder()
.setCount(count).build();
} catch (IOException ioe) {
CoprocessorRpcUtils.setControllerException(controller, ioe);
Expand Down
15 changes: 15 additions & 0 deletions hbase-protocol-shaded/README.txt
Original file line number Diff line number Diff line change
Expand Up @@ -4,3 +4,18 @@ overlap with protos that are used by coprocessor endpoints
a different name, the generated classes are relocated
-- i.e. shaded -- to a new location; they are moved from
org.apache.hadoop.hbase.* to org.apache.hadoop.hbase.shaded.

proto files layout:
protobuf/client - client to server messages, client rpc service and protos, used in hbase-client exclusively;
protobuf/rest - hbase-rest messages;
protobuf/rpc - rpc and post-rpc tracing messages;
protobuf/server/coprocessor - coprocessor rpc services;
protobuf/server/coprocessor/example - coprocessors rpc services examples from hbase-examples;
protobuf/server/io - filesystem and hbase-server/io protos;
protobuf/server/maser - master rpc services and messages;
protobuf/server/region - region rpc services and messages (except client rpc service, which is in Client.proto);
protobuf/server/rsgroup - rsgroup protos;
protobuf/server/zookeeper - protos for zookeeper and ones used exclusively in hbase-zookeeper module;
protobuf/server - protos used across other server protos;
protobuf/test - protos used in tests;
protobuf/ - protos used across other protos, exclusive for hbase-mapreduce and hbase-backup, other protos.
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,9 @@ option java_generate_equals_and_hash = true;
option optimize_for = SPEED;

import "HBase.proto";
import "Filter.proto";
import "client/Filter.proto";
import "Cell.proto";
import "Comparator.proto";
import "client/Comparator.proto";
import "MapReduce.proto";

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ option java_generate_equals_and_hash = true;
option optimize_for = SPEED;

import "HBase.proto";
import "Comparator.proto";
import "client/Comparator.proto";

message Filter {
required string name = 1;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
syntax = "proto2";
package org.apache.hadoop.hbase.shaded.rest.protobuf.generated;

import "CellMessage.proto";
import "rest/CellMessage.proto";

message CellSet {
message Row {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,5 +19,5 @@ syntax = "proto2";
package org.apache.hadoop.hbase.shaded.rest.protobuf.generated;

message Namespaces {
repeated string namespace = 1;
repeated string namespace = 1;
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/
syntax = "proto2";
import "ColumnSchemaMessage.proto";
import "rest/ColumnSchemaMessage.proto";

package org.apache.hadoop.hbase.shaded.rest.protobuf.generated;

Expand All @@ -25,7 +25,7 @@ message TableSchema {
message Attribute {
required string name = 1;
required string value = 2;
}
}
repeated Attribute attrs = 2;
repeated ColumnSchema columns = 3;
// optional helpful encodings of commonly used attributes
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
syntax = "proto2";
package hbase.pb;

import "Tracing.proto";
import "rpc/Tracing.proto";
import "HBase.proto";

option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,9 @@ 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
//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
//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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@ option java_generate_equals_and_hash = true;
option optimize_for = SPEED;

import "HBase.proto";
import "ClusterId.proto";
import "FS.proto";
import "server/ClusterId.proto";
import "server/io/FS.proto";

message RegionState {
required RegionInfo region_info = 1;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ option java_generate_equals_and_hash = true;
option optimize_for = SPEED;

import "google/protobuf/any.proto";
import "ErrorHandling.proto";
import "server/ErrorHandling.proto";

enum ProcedureState {
INITIALIZING = 1; // Procedure in construction, not yet added to the executor
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,9 @@ option java_generic_services = true;
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;

import "AccessControl.proto";
import "FS.proto";
import "HBase.proto";
import "server/io/FS.proto";
import "server/coprocessor/AccessControl.proto";

/**
* Description of the snapshot to take
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ option java_generic_services = true;
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;

import "Client.proto";
import "client/Client.proto";

message AggregateRequest {
/** The request passed to the AggregateService consists of three parts
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
option java_generic_services = true;

import "Client.proto";
import "client/Client.proto";

service ExportService {
rpc export (ExportRequest) returns (ExportResponse);
Expand Down
Loading

0 comments on commit 740e937

Please sign in to comment.