Skip to content

Commit

Permalink
Use our own protoc-gen binary
Browse files Browse the repository at this point in the history
This is a convenient place to put global configuration of our protos
without polluting the .proto files.
  • Loading branch information
tamird committed Dec 9, 2015
1 parent 1fde111 commit daf39dd
Show file tree
Hide file tree
Showing 34 changed files with 93 additions and 152 deletions.
2 changes: 1 addition & 1 deletion GLOCKFILE
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
cmd github.com/cockroachdb/c-protobuf/cmd/protoc
cmd github.com/cockroachdb/cockroach/protoc-gen-gogoroach
cmd github.com/cockroachdb/yacc
cmd github.com/gogo/protobuf/protoc-gen-gogo
cmd github.com/golang/lint/golint
cmd github.com/gordonklaus/ineffassign
cmd github.com/jteeuwen/go-bindata/go-bindata
Expand Down
4 changes: 4 additions & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -178,6 +178,10 @@ clean:
find . -name '*.test' -type f -exec rm -f {} \;
rm -f .bootstrap

.PHONY: protobuf
protobuf:
$(MAKE) -C .. -f cockroach/build/protobuf.mk

ifneq ($(SKIP_BOOTSTRAP),1)

GITHOOKS := $(subst githooks/,.git/hooks/,$(wildcard githooks/*))
Expand Down
10 changes: 5 additions & 5 deletions build/protobuf.mk
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,13 @@ GOGOPROTO_ROOT := $(GITHUB_ROOT)/gogo/protobuf
NATIVE_ROOT := $(REPO_ROOT)/storage/engine/rocksdb

# Ensure we have an unambiguous GOPATH
GOPATH := $(GITHUB_ROOT)/../..
# ^ ^~ GOPATH
# |~ GOPATH/src
GOPATH := $(realpath $(GITHUB_ROOT)/../..)
# ^ ^~ GOPATH
# |~ GOPATH/src

GOPATH_BIN := $(GOPATH)/bin
PROTOC := $(GOPATH_BIN)/protoc
PLUGIN_SUFFIX := gogo
PLUGIN_SUFFIX := gogoroach
PROTOC_PLUGIN := $(GOPATH_BIN)/protoc-gen-$(PLUGIN_SUFFIX)
GOGOPROTO_PROTO := $(GOGOPROTO_ROOT)/gogoproto/gogo.proto
GOGOPROTO_PATH := $(GOGOPROTO_ROOT):$(GOGOPROTO_ROOT)/protobuf
Expand All @@ -56,7 +56,7 @@ protos: $(GO_SOURCES) $(CPP_HEADERS) $(CPP_SOURCES) $(ENGINE_CPP_HEADERS) $(ENGI
REPO_NAME := cockroachdb
IMPORT_PREFIX := github.com/$(REPO_NAME)/

$(GO_SOURCES): $(PROTOC) $(GO_PROTOS) $(GOGOPROTO_PROTO) $(PROTOC_PLUGIN)
$(GO_SOURCES): $(PROTOC) $(GO_PROTOS) $(GOGOPROTO_PROTO)
find $(REPO_ROOT) -not -path '*/.*' -name *.pb.go | xargs rm
for dir in $(sort $(dir $(GO_PROTOS))); do \
$(PROTOC) -I.:$(GOGOPROTO_PATH):$(COREOS_PATH) --plugin=$(PROTOC_PLUGIN) --$(PLUGIN_SUFFIX)_out=import_prefix=$(IMPORT_PREFIX):$(ORG_ROOT) $$dir/*.proto; \
Expand Down
6 changes: 0 additions & 6 deletions config/config.proto
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,6 @@ import "cockroach/roachpb/metadata.proto";
import "cockroach/roachpb/data.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// GCPolicy defines garbage collection policies which apply to MVCC
// values within a zone.
//
Expand Down
6 changes: 0 additions & 6 deletions gossip/gossip.proto
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,6 @@ import "cockroach/roachpb/metadata.proto";
import "cockroach/util/unresolved_addr.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// Request is the request struct passed with the Gossip RPC.
message Request {
// Requesting node's ID.
Expand Down
2 changes: 1 addition & 1 deletion main.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package main

//go:generate make -C .. -f cockroach/build/protobuf.mk
//go:generate make protobuf

import (
"fmt"
Expand Down
6 changes: 0 additions & 6 deletions multiraft/rpc.proto
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,6 @@ import "cockroach/roachpb/metadata.proto";
import "etcd/raft/raftpb/raft.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// RaftMessageRequest is the request used to send raft messages using our
// protobuf-based RPC codec.
message RaftMessageRequest {
Expand Down
74 changes: 74 additions & 0 deletions protoc-gen-gogoroach/main.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
package main

import (
"github.com/gogo/protobuf/protoc-gen-gogo/descriptor"
"github.com/gogo/protobuf/vanity"
"github.com/gogo/protobuf/vanity/command"
)

func main() {
req := command.Read()
files := req.GetProtoFile()
files = vanity.FilterFiles(files, vanity.NotInPackageGoogleProtobuf)

for _, opt := range []func(*descriptor.FileDescriptorProto){
vanity.TurnOffGoGettersAll,

// Currently harms readability.
// vanity.TurnOffGoEnumPrefixAll,

// `String() string` is part of gogoproto.Message, so we need this.
// vanity.TurnOffGoStringerAll,

// Maybe useful for tests? Not using currently.
// vanity.TurnOnVerboseEqualAll,

// Incompatible with oneof, and also not sure what the value is.
// vanity.TurnOnFaceAll,

// Requires that all child messages are generated with this, which
// is not the case for Raft messages which wrap raftpb (which
// doesn't use this).
// vanity.TurnOnGoStringAll,

// Not useful for us.
// vanity.TurnOnPopulateAll,

// Conflicts with `GoStringerAll`, which is enabled.
// vanity.TurnOnStringerAll,

// This generates a method that takes `interface{}`, which sucks.
// vanity.TurnOnEqualAll,

// Not useful for us.
// vanity.TurnOnDescriptionAll,
// vanity.TurnOnTestGenAll,
// vanity.TurnOnBenchGenAll,

vanity.TurnOnMarshalerAll,
vanity.TurnOnUnmarshalerAll,
vanity.TurnOnSizerAll,

// Enabling these causes `String() string` on Enums to be inlined.
// Not worth it.
// vanity.TurnOffGoEnumStringerAll,
// vanity.TurnOnEnumStringerAll,

// Not clear that this is worthwhile.
// vanity.TurnOnUnsafeUnmarshalerAll,
// vanity.TurnOnUnsafeMarshalerAll,

// Something something extensions; we don't use 'em currently.
// vanity.TurnOffGoExtensionsMapAll,

vanity.TurnOffGoUnrecognizedAll,

// Adds unnecessary dependency on golang/protobuf.
// vanity.TurnOffGogoImport,
} {
vanity.ForEachFile(files, opt)
}

resp := command.Generate(req)
command.Write(resp)
}
7 changes: 1 addition & 6 deletions roachpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,6 @@ import "cockroach/roachpb/data.proto";
import "cockroach/roachpb/errors.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// ReadConsistencyType specifies what type of consistency is observed
// during read operations.
enum ReadConsistencyType {
Expand Down Expand Up @@ -354,6 +348,7 @@ message GCResponse {
// TxnPushType determines what action to take when pushing a transaction.
enum PushTxnType {
option (gogoproto.goproto_enum_prefix) = false;

// Push the timestamp forward if possible to accommodate a concurrent reader.
PUSH_TIMESTAMP = 0;
// Abort the transaction if possible to accommodate a concurrent writer.
Expand Down
6 changes: 0 additions & 6 deletions roachpb/data.proto
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,6 @@ option go_package = "roachpb";
import "cockroach/roachpb/metadata.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// Span is supplied with every storage node request.
message Span {
// The key for request. If the request operates on a range, this
Expand Down
6 changes: 0 additions & 6 deletions roachpb/errors.proto
Original file line number Diff line number Diff line change
Expand Up @@ -29,12 +29,6 @@ import weak "gogoproto/gogo.proto";
// `String() string`.
// option (gogoproto.goproto_stringer_all) = false;

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// A NotLeaderError indicates that the current range is not the
// leader. If the leader is known, its Replica is set in the error.
message NotLeaderError {
Expand Down
6 changes: 0 additions & 6 deletions roachpb/internal.proto
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,6 @@ import "cockroach/roachpb/api.proto";
import "cockroach/roachpb/metadata.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// A RaftCommand is a command which can be serialized and sent via
// raft.
message RaftCommand {
Expand Down
6 changes: 0 additions & 6 deletions roachpb/metadata.proto
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,6 @@ option go_package = "roachpb";
import "cockroach/util/unresolved_addr.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// Attributes specifies a list of arbitrary strings describing
// node topology, store type, and machine capabilities.
message Attributes {
Expand Down
6 changes: 0 additions & 6 deletions rpc/codec/message/arith.proto
Original file line number Diff line number Diff line change
Expand Up @@ -8,12 +8,6 @@ option go_package = "message";

import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

message ArithRequest {
optional int32 a = 1 [(gogoproto.nullable) = false];
optional int32 b = 2 [(gogoproto.nullable) = false];
Expand Down
6 changes: 0 additions & 6 deletions rpc/codec/message/echo.proto
Original file line number Diff line number Diff line change
Expand Up @@ -8,12 +8,6 @@ option go_package = "message";

import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

message EchoRequest {
optional string msg = 1 [(gogoproto.nullable) = false];
}
Expand Down
6 changes: 0 additions & 6 deletions rpc/codec/wire/wire.proto
Original file line number Diff line number Diff line change
Expand Up @@ -47,12 +47,6 @@ option go_package = "wire";

import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

enum CompressionType {
NONE = 0;
SNAPPY = 1;
Expand Down
6 changes: 0 additions & 6 deletions rpc/heartbeat.proto
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,6 @@ option go_package = "rpc";

import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// RemoteOffset keeps track of this client's estimate of its offset from a
// remote server. Uncertainty is the maximum error in the reading of this
// offset, so that the real offset should be in the interval
Expand Down
6 changes: 0 additions & 6 deletions server/status/status.proto
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,6 @@ import "cockroach/roachpb/metadata.proto";
import "cockroach/storage/engine/mvcc.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// NodeStatus contains the stats needed to calculate the current status of a
// node.
message NodeStatus {
Expand Down
6 changes: 0 additions & 6 deletions sql/driver/wire.proto
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,6 @@ option go_package = "driver";

import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

message Datum {
// Timestamp represents an absolute timestamp devoid of time-zone.
message Timestamp {
Expand Down
6 changes: 0 additions & 6 deletions sql/privilege.proto
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,6 @@ option go_package = "sql";

import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// UserPrivileges describes the list of privileges available for a given user.
message UserPrivileges {
optional string user = 1 [(gogoproto.nullable) = false];
Expand Down
6 changes: 0 additions & 6 deletions sql/session.proto
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,6 @@ import "cockroach/roachpb/data.proto";
import "cockroach/sql/driver/wire.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

message Session {
optional string database = 1 [(gogoproto.nullable) = false];
optional int32 syntax = 2 [(gogoproto.nullable) = false];
Expand Down
6 changes: 0 additions & 6 deletions sql/structured.proto
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,6 @@ import "cockroach/roachpb/data.proto";
import "cockroach/sql/privilege.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

message ColumnType {
// These mirror the types supported by the sql/parser. See
// sql/parser/types.go.
Expand Down
6 changes: 0 additions & 6 deletions storage/engine/mvcc.proto
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,6 @@ option go_package = "engine";
import "cockroach/roachpb/data.proto";
import weak "gogoproto/gogo.proto";

option (gogoproto.goproto_getters_all) = false;
option (gogoproto.goproto_unrecognized_all) = false;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;

// MVCCMetadata holds MVCC metadata for a key. Used by storage/engine/mvcc.go.
message MVCCMetadata {
optional roachpb.Transaction txn = 1;
Expand Down
3 changes: 1 addition & 2 deletions storage/engine/rocksdb/cockroach/roachpb/api.pb.cc

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

3 changes: 1 addition & 2 deletions storage/engine/rocksdb/cockroach/roachpb/data.pb.cc

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Loading

0 comments on commit daf39dd

Please sign in to comment.