You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by el...@apache.org on 2018/09/21 15:15:12 UTC

[1/3] incubator-ratis git commit: RATIS-316. Use ratis-thirdparty and reorganize generated code

Repository: incubator-ratis
Updated Branches:
  refs/heads/master 523fb6383 -> 3b9d50ded


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto/src/main/proto/Raft.proto
----------------------------------------------------------------------
diff --git a/ratis-proto/src/main/proto/Raft.proto b/ratis-proto/src/main/proto/Raft.proto
new file mode 100644
index 0000000..535914d
--- /dev/null
+++ b/ratis-proto/src/main/proto/Raft.proto
@@ -0,0 +1,307 @@
+/**
+ * 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 = "proto3";
+option java_package = "org.apache.ratis.proto";
+option java_outer_classname = "RaftProtos";
+option java_generate_equals_and_hash = true;
+package ratis.common;
+
+message RaftPeerProto {
+  bytes id = 1;      // id of the peer
+  string address = 2; // e.g. IP address, hostname etc.
+}
+
+message RaftGroupIdProto {
+  bytes id = 1;
+}
+
+message RaftGroupProto {
+  RaftGroupIdProto groupId = 1;
+  repeated RaftPeerProto peers = 2;
+}
+
+message RaftConfigurationProto {
+  repeated RaftPeerProto peers = 1; // the peers in the current or new conf
+  repeated RaftPeerProto oldPeers = 2; // the peers in the old conf
+}
+
+message SMLogEntryProto {
+  // TODO: This is not super efficient if the SM itself uses PB to serialize its own data for a
+  // log entry. Data will be copied twice. We should directly support having any Message from SM
+  bytes data = 1;
+
+  bytes stateMachineData = 2; // State machine specific data which is not written to log.
+  bool stateMachineDataAttached = 3; // set this flag when state machine data is attached.
+  uint64 serializedProtobufSize = 4; // size of the serialized LogEntryProto along with stateMachineData
+}
+
+message LeaderNoOp {
+  // empty
+}
+
+message LogEntryProto {
+  uint64 term = 1;
+  uint64 index = 2;
+
+  oneof LogEntryBody {
+    SMLogEntryProto smLogEntry = 3;
+    RaftConfigurationProto configurationEntry = 4;
+    LeaderNoOp noOp = 5;
+  }
+
+  // clientId and callId are used to rebuild the retry cache. They're not
+  // necessary for configuration change since re-conf is idempotent.
+  bytes clientId = 6;
+  uint64 callId = 7;
+}
+
+message TermIndexProto {
+  uint64 term = 1;
+  uint64 index = 2;
+}
+
+message RaftRpcRequestProto {
+  bytes requestorId = 1;
+  bytes replyId = 2;
+  RaftGroupIdProto raftGroupId = 3;
+  uint64 callId = 4;
+
+  uint64 seqNum = 15;
+}
+
+message RaftRpcReplyProto {
+  bytes requestorId = 1;
+  bytes replyId = 2;
+  RaftGroupIdProto raftGroupId = 3;
+  uint64 callId = 4;
+
+  bool success = 15;
+}
+
+message FileChunkProto {
+  string filename = 1; // relative to root
+  uint64 totalSize = 2;
+  bytes fileDigest = 3;
+  uint32 chunkIndex = 4;
+  uint64 offset = 5;
+  bytes data = 6;
+  bool done = 7;
+}
+
+enum InstallSnapshotResult {
+  SUCCESS = 0;
+  NOT_LEADER = 1;
+}
+
+message RequestVoteRequestProto {
+  RaftRpcRequestProto serverRequest = 1;
+  uint64 candidateTerm = 2;
+  TermIndexProto candidateLastEntry = 3;
+}
+
+message RequestVoteReplyProto {
+  RaftRpcReplyProto serverReply = 1;
+  uint64 term = 2;
+  bool shouldShutdown = 3;
+}
+
+message CommitInfoProto {
+  RaftPeerProto server = 1;
+  uint64 commitIndex = 2;
+}
+
+message AppendEntriesRequestProto {
+  RaftRpcRequestProto serverRequest = 1;
+  uint64 leaderTerm = 2;
+  TermIndexProto previousLog = 3;
+  repeated LogEntryProto entries = 4;
+  uint64 leaderCommit = 5;
+  bool initializing = 6;
+
+  repeated CommitInfoProto commitInfos = 15;
+}
+
+message AppendEntriesReplyProto {
+  enum AppendResult {
+    SUCCESS = 0;
+    NOT_LEADER = 1; // the requester's term is not large enough
+    INCONSISTENCY = 2; // gap between the local log and the entries
+  }
+
+  RaftRpcReplyProto serverReply = 1;
+  uint64 term = 2;
+  uint64 nextIndex = 3;
+  AppendResult result = 4;
+}
+
+message InstallSnapshotRequestProto {
+  RaftRpcRequestProto serverRequest = 1;
+  string requestId = 2; // an identifier for chunked-requests.
+  uint32 requestIndex = 3; // the index for this request chunk. Starts from 0.
+  RaftConfigurationProto raftConfiguration = 4;
+  uint64 leaderTerm = 5;
+  TermIndexProto termIndex = 6;
+  repeated FileChunkProto fileChunks = 7;
+  uint64 totalSize = 8;
+  bool done = 9; // whether this is the final chunk for the same req.
+}
+
+message InstallSnapshotReplyProto {
+  RaftRpcReplyProto serverReply = 1;
+  uint32 requestIndex = 2;
+  uint64 term = 3;
+  InstallSnapshotResult result = 4;
+}
+
+message ClientMessageEntryProto {
+  bytes content = 1;
+}
+
+enum ReplicationLevel {
+  MAJORITY = 0;
+  ALL = 1;
+}
+
+
+/** Role of raft peer */
+enum RaftPeerRole {
+  LEADER = 0;
+  CANDIDATE = 1;
+  FOLLOWER = 2;
+}
+
+message WriteRequestTypeProto {
+  ReplicationLevel replication = 1;
+}
+
+message ReadRequestTypeProto {
+}
+
+message StaleReadRequestTypeProto {
+  uint64 minIndex = 1;
+}
+
+// normal client request
+message RaftClientRequestProto {
+  RaftRpcRequestProto rpcRequest = 1;
+  ClientMessageEntryProto message = 2;
+
+  oneof Type {
+    WriteRequestTypeProto write = 3;
+    ReadRequestTypeProto read = 4;
+    StaleReadRequestTypeProto staleRead = 5;
+  }
+}
+
+message NotLeaderExceptionProto {
+  RaftPeerProto suggestedLeader = 1;
+  repeated RaftPeerProto peersInConf = 2;
+}
+
+message NotReplicatedExceptionProto {
+  uint64 callId = 1;
+  ReplicationLevel replication = 2;
+  uint64 logIndex = 3;
+}
+
+message StateMachineExceptionProto {
+  string exceptionClassName = 1;
+  string errorMsg = 2;
+  bytes stacktrace = 3;
+}
+
+message RaftClientReplyProto {
+  RaftRpcReplyProto rpcReply = 1;
+  ClientMessageEntryProto message = 2;
+
+  oneof ExceptionDetails {
+    NotLeaderExceptionProto notLeaderException = 3;
+    NotReplicatedExceptionProto notReplicatedException = 4;
+    StateMachineExceptionProto stateMachineException = 5;
+  }
+
+  repeated CommitInfoProto commitInfos = 15;
+}
+
+// setConfiguration request
+message SetConfigurationRequestProto {
+  RaftRpcRequestProto rpcRequest = 1;
+  repeated RaftPeerProto peers = 2;
+}
+
+// A request to add a new group
+message GroupAddRequestProto {
+  RaftGroupProto group = 1; // the group to be added.
+}
+
+message GroupRemoveRequestProto {
+  RaftGroupIdProto groupId = 1; // the group to be removed.
+  bool deleteDirectory = 2; // delete the directory for that group?
+}
+
+message GroupManagementRequestProto {
+  RaftRpcRequestProto rpcRequest = 1;
+
+  oneof Op {
+    GroupAddRequestProto groupAdd = 2;
+    GroupRemoveRequestProto groupRemove = 3;
+  }
+}
+
+// server info request
+message ServerInformationRequestProto {
+  RaftRpcRequestProto rpcRequest = 1;
+}
+
+message ServerRpcProto {
+  RaftPeerProto id = 1;
+  uint64 lastRpcElapsedTimeMs = 2;
+}
+
+message LeaderInfoProto {
+  repeated ServerRpcProto followerInfo = 1;
+}
+
+message FollowerInfoProto {
+  ServerRpcProto leaderInfo = 1;
+  bool inLogSync = 2;
+}
+
+message CandidateInfoProto {
+  uint64 lastLeaderElapsedTimeMs = 1;
+}
+
+message RoleInfoProto {
+  RaftPeerProto self = 1;
+  RaftPeerRole role = 2;
+  uint64 roleElapsedTimeMs = 3;
+
+  oneof PeerInfo {
+    LeaderInfoProto leaderInfo = 4;
+    FollowerInfoProto followerInfo = 5;
+    CandidateInfoProto candidateInfo = 6;
+  }
+}
+
+message ServerInformationReplyProto {
+  RaftRpcReplyProto rpcReply = 1;
+  RaftGroupProto group = 2;
+  RoleInfoProto role = 3;
+  bool isRaftStorageHealthy = 4;
+  repeated CommitInfoProto commitInfos = 5;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-replicated-map/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-replicated-map/pom.xml b/ratis-replicated-map/pom.xml
index 63fcad7..1eda796 100644
--- a/ratis-replicated-map/pom.xml
+++ b/ratis-replicated-map/pom.xml
@@ -25,7 +25,11 @@
 
   <dependencies>
     <dependency>
-      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-proto</artifactId>
       <groupId>org.apache.ratis</groupId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml
index 14acf77..1bc4e77 100644
--- a/ratis-server/pom.xml
+++ b/ratis-server/pom.xml
@@ -25,7 +25,11 @@
 
   <dependencies>
     <dependency>
-      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-proto</artifactId>
       <groupId>org.apache.ratis</groupId>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/CommitInfoCache.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/CommitInfoCache.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/CommitInfoCache.java
index e74ff9c..7b8cd6c 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/CommitInfoCache.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/CommitInfoCache.java
@@ -19,7 +19,7 @@ package org.apache.ratis.server.impl;
 
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.shaded.proto.RaftProtos.CommitInfoProto;
+import org.apache.ratis.proto.RaftProtos.CommitInfoProto;
 import org.apache.ratis.util.ProtoUtils;
 
 import java.util.Objects;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
index 8c49005..c60352d 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
@@ -20,8 +20,8 @@ package org.apache.ratis.server.impl;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.util.Daemon;
 import org.apache.ratis.util.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
index 839aa69..e355d4f 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
@@ -22,9 +22,9 @@ import org.apache.ratis.protocol.*;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.shaded.proto.RaftProtos.CommitInfoProto;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.CommitInfoProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.*;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
index 58f5525..c237917 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
@@ -26,8 +26,8 @@ import org.apache.ratis.server.storage.RaftLog.EntryWithData;
 import org.apache.ratis.server.storage.FileInfo;
 import org.apache.ratis.server.storage.RaftLog;
 import org.apache.ratis.server.storage.RaftLogIOException;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.proto.RaftProtos.*;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.util.*;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
index cdb283f..a95184a 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
@@ -19,7 +19,7 @@ package org.apache.ratis.server.impl;
 
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.server.impl.RetryCache.CacheEntry;
-import org.apache.ratis.shaded.proto.RaftProtos.ReplicationLevel;
+import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
index 92b3e96..c84f944 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
@@ -19,8 +19,8 @@ package org.apache.ratis.server.impl;
 
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.server.impl.RetryCache.CacheEntry;
-import org.apache.ratis.shaded.proto.RaftProtos.ReplicationLevel;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
+import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index 66603a2..9de81ff 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -28,8 +28,8 @@ import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.FileInfo;
 import org.apache.ratis.server.storage.RaftLog;
 import org.apache.ratis.server.storage.RaftStorageDirectory;
-import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.ratis.proto.RaftProtos.*;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.TransactionContext;
@@ -47,9 +47,9 @@ import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import static org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
-import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
-import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.SMLOGENTRY;
+import static org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
+import static org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+import static org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.SMLOGENTRY;
 import static org.apache.ratis.util.LifeCycle.State.*;
 
 public class RaftServerImpl implements RaftServerProtocol, RaftServerAsynchronousProtocol,

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
index 2a119b2..f12905c 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
@@ -25,7 +25,7 @@ import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.RaftServerRpc;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.proto.RaftProtos.*;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.CheckedFunction;
 import org.apache.ratis.util.IOUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
index 81ce9ae..887f39c 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
@@ -24,9 +24,9 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.protocol.RaftClientReply;
-import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.ratis.shaded.com.google.common.cache.Cache;
-import org.apache.ratis.shaded.com.google.common.cache.CacheBuilder;
+import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.ratis.thirdparty.com.google.common.cache.Cache;
+import org.apache.ratis.thirdparty.com.google.common.cache.CacheBuilder;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.TimeDuration;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java
index 20ec951..9ee2aa1 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java
@@ -18,7 +18,7 @@
 
 package org.apache.ratis.server.impl;
 
-import org.apache.ratis.shaded.proto.RaftProtos.RaftPeerRole;
+import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
 import org.apache.ratis.util.Timestamp;
 
 import java.util.concurrent.atomic.AtomicReference;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
index 3774737..518db7c 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
@@ -28,9 +28,9 @@ import java.util.stream.Collectors;
 import org.apache.ratis.client.impl.ClientProtoUtils;
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto.*;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.proto.RaftProtos.*;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.*;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
index 3f064a8..86b6b00 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
@@ -22,8 +22,8 @@ import org.apache.ratis.protocol.*;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.*;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.TransactionContext;
@@ -39,7 +39,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Consumer;
 
 import static org.apache.ratis.server.impl.RaftServerImpl.LOG;
-import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+import static org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
 
 /**
  * Common states of a raft peer. Protected by RaftServer's lock.
@@ -379,4 +379,4 @@ public class ServerState implements Closeable {
   public long getLastAppliedIndex() {
     return stateMachineUpdater.getLastAppliedIndex();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
index c92f3ac..02dcb31 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
@@ -21,7 +21,7 @@ import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.protocol.Message;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java b/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java
index d9461c7..d13f7f5 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java
@@ -18,8 +18,8 @@
 
 package org.apache.ratis.server.protocol;
 
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
 
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java b/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java
index f1f5512..d1ef4f3 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java
@@ -19,12 +19,12 @@ package org.apache.ratis.server.protocol;
 
 import java.io.IOException;
 
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto;
 
 public interface RaftServerProtocol {
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
index 0edea13..22d7949 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
@@ -24,7 +24,7 @@ import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
index dedfe7e..a09b90d 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
@@ -18,8 +18,8 @@
 package org.apache.ratis.server.storage;
 
 import org.apache.ratis.server.impl.RaftServerConstants;
-import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.PureJavaCrc32C;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
index 48a00e1..ce149dd 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
@@ -19,9 +19,9 @@ package org.apache.ratis.server.storage;
 
 import org.apache.ratis.protocol.ChecksumException;
 import org.apache.ratis.server.impl.RaftServerConstants;
-import org.apache.ratis.shaded.com.google.protobuf.CodedInputStream;
-import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.CodedInputStream;
+import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.PureJavaCrc32C;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
index 2550855..abf92e5 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
@@ -19,10 +19,10 @@ package org.apache.ratis.server.storage;
 
 import org.apache.ratis.server.impl.ServerProtoUtils;
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.ratis.shaded.com.google.common.cache.CacheLoader;
-import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.ratis.thirdparty.com.google.common.cache.CacheLoader;
+import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
index 7b3b1b9..71bed5c 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
@@ -22,7 +22,7 @@ import org.apache.ratis.server.impl.RaftConfiguration;
 import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.server.impl.ServerProtoUtils;
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.AutoCloseableLock;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
index 296dce6..30d5c8f 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
@@ -25,7 +25,7 @@ import org.apache.ratis.server.impl.RaftConfiguration;
 import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.server.impl.ServerProtoUtils;
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.AutoCloseableLock;
 import org.apache.ratis.util.JavaUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
index e1ff84b..0cb1047 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
@@ -25,7 +25,7 @@ import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.CacheInvalidationPolicy.CacheInvalidationPolicyDefault;
 import org.apache.ratis.server.storage.LogSegment.LogRecord;
 import org.apache.ratis.server.storage.RaftStorageDirectory.LogPathAndIndex;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
index 68f303d..c0d1cb9 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
@@ -29,7 +29,7 @@ import org.apache.ratis.server.impl.ServerProtoUtils;
 import org.apache.ratis.server.storage.RaftLogCache.SegmentFileInfo;
 import org.apache.ratis.server.storage.RaftLogCache.TruncationSegments;
 import org.apache.ratis.server.storage.SegmentedRaftLog.Task;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.*;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
index 9df16f8..b7cf920 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
@@ -27,7 +27,7 @@ import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.LogSegment.LogRecord;
 import org.apache.ratis.server.storage.LogSegment.LogRecordWithEntry;
 import org.apache.ratis.server.storage.RaftStorageDirectory.LogPathAndIndex;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.AutoCloseableLock;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
index ccccf22..77a9963 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
@@ -24,8 +24,8 @@ import java.nio.channels.FileChannel;
 
 import org.apache.ratis.io.MD5Hash;
 import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.shaded.proto.RaftProtos.FileChunkProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.proto.RaftProtos.FileChunkProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.FileUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachine.java
index c76b2e4..6c8b2df 100644
--- a/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachine.java
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachine.java
@@ -26,8 +26,8 @@ import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RaftConfiguration;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.RaftStorage;
-import org.apache.ratis.shaded.proto.RaftProtos.RoleInfoProto;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.RoleInfoProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.LifeCycle;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/statemachine/TransactionContext.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/TransactionContext.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/TransactionContext.java
index be92532..a112772 100644
--- a/ratis-server/src/main/java/org/apache/ratis/statemachine/TransactionContext.java
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/TransactionContext.java
@@ -18,10 +18,10 @@
 package org.apache.ratis.statemachine;
 
 import org.apache.ratis.protocol.RaftClientRequest;
-import org.apache.ratis.shaded.proto.RaftProtos;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase;
-import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase;
+import org.apache.ratis.proto.RaftProtos.SMLogEntryProto;
 
 import java.io.IOException;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java
index 16b4f6f..f43fdf8 100644
--- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java
@@ -27,7 +27,7 @@ import org.apache.ratis.server.impl.RaftConfiguration;
 import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.RaftStorage;
-import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.proto.RaftProtos.SMLogEntryProto;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.StateMachineStorage;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java
index b49a443..5216cff 100644
--- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java
@@ -20,10 +20,10 @@ package org.apache.ratis.statemachine.impl;
 import java.io.IOException;
 import java.util.Objects;
 import org.apache.ratis.protocol.RaftClientRequest;
-import org.apache.ratis.shaded.proto.RaftProtos;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase;
-import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase;
+import org.apache.ratis.proto.RaftProtos.SMLogEntryProto;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
index 0740956..d2aef5b 100644
--- a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
+++ b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
@@ -29,7 +29,7 @@ import org.apache.ratis.server.impl.RaftServerProxy;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
 import org.apache.ratis.server.storage.MemoryRaftLog;
 import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.shaded.proto.RaftProtos.ReplicationLevel;
+import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
 import org.apache.ratis.statemachine.impl.BaseStateMachine;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java
index 3d80f5e..776e1f1 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java
@@ -27,11 +27,11 @@ import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.impl.RaftServerProxy;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.ratis.shaded.proto.RaftProtos.CommitInfoProto;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.shaded.proto.RaftProtos.ReplicationLevel;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.ratis.proto.RaftProtos.CommitInfoProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
 import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.JavaUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
index 8a41d90..9cd7b7a 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
@@ -32,8 +32,8 @@ import org.apache.ratis.server.impl.RaftServerProxy;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
 import org.apache.ratis.server.impl.RetryCacheTestUtil;
 import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.shaded.proto.RaftProtos.ReplicationLevel;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
 import org.apache.ratis.util.ExitUtils;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.LogUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
index e167f17..fa529b6 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
@@ -30,9 +30,9 @@ import org.apache.ratis.server.impl.RaftServerProxy;
 import org.apache.ratis.server.impl.ServerProtoUtils;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.SMLogEntryProto;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java b/ratis-server/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java
index 04a2a8d..6ec321f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java
+++ b/ratis-server/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java
@@ -23,7 +23,7 @@ import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.ratis.shaded.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.LogUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java b/ratis-server/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
index 51af8af..b8ef87e 100644
--- a/ratis-server/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
+++ b/ratis-server/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
@@ -24,7 +24,7 @@ import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.ratis.shaded.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.LogUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/protocol/TestRaftId.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/protocol/TestRaftId.java b/ratis-server/src/test/java/org/apache/ratis/protocol/TestRaftId.java
index b454c31..30b7ed5 100644
--- a/ratis-server/src/test/java/org/apache/ratis/protocol/TestRaftId.java
+++ b/ratis-server/src/test/java/org/apache/ratis/protocol/TestRaftId.java
@@ -18,7 +18,7 @@
 package org.apache.ratis.protocol;
 
 import org.apache.ratis.BaseTest;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
index 6374a21..16536e7 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
@@ -47,7 +47,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import static java.util.Arrays.asList;
 import static org.apache.ratis.server.impl.RaftServerTestUtil.waitAndCheckNewConf;
-import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+import static org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
 
 public abstract class RaftReconfigurationBaseTest extends BaseTest {
   static {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java
index 9e63063..b31c7e8 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java
@@ -18,7 +18,7 @@
 package org.apache.ratis.server.impl;
 
 import org.apache.ratis.protocol.ClientId;
-import org.apache.ratis.shaded.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.util.TimeDuration;
 import org.junit.Assert;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerInformationBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerInformationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerInformationBaseTest.java
index 2862e0d..dd440cb 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerInformationBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerInformationBaseTest.java
@@ -23,7 +23,7 @@ import org.apache.ratis.MiniRaftCluster;
 import org.apache.ratis.RaftTestUtil;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.protocol.*;
-import org.apache.ratis.shaded.proto.RaftProtos.CommitInfoProto;
+import org.apache.ratis.proto.RaftProtos.CommitInfoProto;
 import org.apache.ratis.util.LogUtils;
 import org.junit.Assert;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java
index df0545d..d5545cb 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java
@@ -19,9 +19,9 @@ package org.apache.ratis.server.simulation;
 
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.protocol.RaftRpcMessage;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto;
 import org.apache.ratis.util.ProtoUtils;
 
 import java.util.Objects;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerRequest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerRequest.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerRequest.java
index e38296c..1ec791e 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerRequest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerRequest.java
@@ -19,9 +19,9 @@ package org.apache.ratis.server.simulation;
 
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.protocol.RaftRpcMessage;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto;
 import org.apache.ratis.util.ProtoUtils;
 
 class RaftServerRequest implements RaftRpcMessage {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
index d9bbc43..52de86f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
@@ -21,7 +21,7 @@ import org.apache.ratis.protocol.*;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerRpc;
 import org.apache.ratis.server.impl.RaftServerProxy;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.proto.RaftProtos.*;
 import org.apache.ratis.util.Daemon;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.JavaUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java
index cc9d76d..ad8308e 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java
@@ -19,7 +19,7 @@ package org.apache.ratis.server.storage;
 
 import org.apache.log4j.Level;
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.shaded.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.util.AutoCloseableLock;
 import org.apache.ratis.util.LogUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
index c94fb73..124f7b8 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
@@ -29,7 +29,7 @@ import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.impl.ServerState;
 import org.apache.ratis.server.storage.CacheInvalidationPolicy.CacheInvalidationPolicyDefault;
 import org.apache.ratis.server.storage.TestSegmentedRaftLog.SegmentRange;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.ProtoUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
index 42e63ea..eeb7ea0 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
@@ -25,7 +25,7 @@ import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.RaftLogCache.TruncationSegments;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.ProtoUtils;
 import org.junit.Assert;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
index cb512a5..c218179 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
@@ -25,8 +25,8 @@ import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
-import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.ProtoUtils;
 import org.apache.ratis.util.SizeInBytes;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
index 690ad59..ce5a92d 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
@@ -24,8 +24,8 @@ import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
 import org.apache.ratis.server.storage.LogSegment.LogRecordWithEntry;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.SMLogEntryProto;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.ProtoUtils;
 import org.apache.ratis.util.SizeInBytes;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
index a95627f..7b26733 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
@@ -29,7 +29,7 @@ import org.apache.ratis.server.impl.RetryCacheTestUtil;
 import org.apache.ratis.server.impl.RetryCache;
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.LogUtils;
 import org.apache.ratis.util.ProtoUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
index cf0f611..584812e 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
@@ -32,7 +32,7 @@ import org.apache.ratis.server.impl.RaftServerTestUtil;
 import org.apache.ratis.server.storage.RaftLog;
 import org.apache.ratis.server.storage.RaftStorageDirectory;
 import org.apache.ratis.server.storage.RaftStorageDirectory.LogPathAndIndex;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.LogUtils;
 import org.junit.After;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
index b6d6d3f..cafd98d 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
@@ -33,10 +33,10 @@ import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.LogInputStream;
 import org.apache.ratis.server.storage.LogOutputStream;
 import org.apache.ratis.server.storage.RaftStorage;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.proto.RaftProtos.RoleInfoProto;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.proto.RaftProtos.RoleInfoProto;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.SMLogEntryProto;
 import org.apache.ratis.statemachine.impl.BaseStateMachine;
 import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
 import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
index 07039b1..189c9da 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
@@ -28,7 +28,7 @@ import org.apache.ratis.protocol.RaftClientRequest;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.proto.RaftProtos.SMLogEntryProto;
 import org.apache.ratis.statemachine.impl.TransactionContextImpl;
 import org.apache.ratis.util.LogUtils;
 import org.junit.*;


[2/3] incubator-ratis git commit: RATIS-316. Use ratis-thirdparty and reorganize generated code

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStateMachine.java b/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStateMachine.java
index 3444dd5..cdc17f3 100644
--- a/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStateMachine.java
+++ b/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStateMachine.java
@@ -23,8 +23,8 @@ import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.RaftStorage;
-import org.apache.ratis.shaded.proto.RaftProtos;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.StateMachineStorage;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.statemachine.impl.BaseStateMachine;
@@ -32,7 +32,7 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
 import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
 import org.apache.ratis.util.AutoCloseableLock;
 
-import org.apache.ratis.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
 
 import java.io.*;
 import java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-logservice/src/test/java/org/apache/ratis/logservice/api/TestLogMessage.java
----------------------------------------------------------------------
diff --git a/ratis-logservice/src/test/java/org/apache/ratis/logservice/api/TestLogMessage.java b/ratis-logservice/src/test/java/org/apache/ratis/logservice/api/TestLogMessage.java
index fb74d64..d58bdda 100644
--- a/ratis-logservice/src/test/java/org/apache/ratis/logservice/api/TestLogMessage.java
+++ b/ratis-logservice/src/test/java/org/apache/ratis/logservice/api/TestLogMessage.java
@@ -22,8 +22,8 @@ import static org.junit.Assert.assertEquals;
 import org.apache.ratis.logservice.api.LogMessage.Type;
 import org.junit.Test;
 
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
 
 public class TestLogMessage {
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-netty/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml
index 2423b32..b573420 100644
--- a/ratis-netty/pom.xml
+++ b/ratis-netty/pom.xml
@@ -25,9 +25,12 @@
 
   <dependencies>
     <dependency>
-      <artifactId>ratis-proto-shaded</artifactId>
       <groupId>org.apache.ratis</groupId>
-      <scope>provided</scope>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-proto</artifactId>
+      <groupId>org.apache.ratis</groupId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
index ce5021a..373011e 100644
--- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
@@ -17,15 +17,15 @@
  */
 package org.apache.ratis.netty;
 
-import org.apache.ratis.shaded.io.netty.bootstrap.Bootstrap;
-import org.apache.ratis.shaded.io.netty.channel.Channel;
-import org.apache.ratis.shaded.io.netty.channel.ChannelFuture;
-import org.apache.ratis.shaded.io.netty.channel.ChannelInitializer;
-import org.apache.ratis.shaded.io.netty.channel.EventLoopGroup;
-import org.apache.ratis.shaded.io.netty.channel.socket.SocketChannel;
-import org.apache.ratis.shaded.io.netty.channel.socket.nio.NioSocketChannel;
-import org.apache.ratis.shaded.io.netty.handler.logging.LogLevel;
-import org.apache.ratis.shaded.io.netty.handler.logging.LoggingHandler;
+import org.apache.ratis.thirdparty.io.netty.bootstrap.Bootstrap;
+import org.apache.ratis.thirdparty.io.netty.channel.Channel;
+import org.apache.ratis.thirdparty.io.netty.channel.ChannelFuture;
+import org.apache.ratis.thirdparty.io.netty.channel.ChannelInitializer;
+import org.apache.ratis.thirdparty.io.netty.channel.EventLoopGroup;
+import org.apache.ratis.thirdparty.io.netty.channel.socket.SocketChannel;
+import org.apache.ratis.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.ratis.thirdparty.io.netty.handler.logging.LogLevel;
+import org.apache.ratis.thirdparty.io.netty.handler.logging.LoggingHandler;
 import org.apache.ratis.util.LifeCycle;
 import org.apache.ratis.util.NetUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
index 20521b6..1c83629 100644
--- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
@@ -18,16 +18,16 @@
 package org.apache.ratis.netty;
 
 import org.apache.ratis.protocol.RaftPeer;
-import org.apache.ratis.shaded.io.netty.channel.*;
-import org.apache.ratis.shaded.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.ratis.shaded.io.netty.channel.socket.SocketChannel;
-import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
-import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufEncoder;
-import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcRequestProto;
-import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto;
-import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
+import org.apache.ratis.thirdparty.io.netty.channel.*;
+import org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.ratis.thirdparty.io.netty.channel.socket.SocketChannel;
+import org.apache.ratis.thirdparty.io.netty.handler.codec.protobuf.ProtobufDecoder;
+import org.apache.ratis.thirdparty.io.netty.handler.codec.protobuf.ProtobufEncoder;
+import org.apache.ratis.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import org.apache.ratis.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
+import org.apache.ratis.proto.RaftProtos.RaftRpcRequestProto;
+import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerReplyProto;
+import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.PeerProxyMap;
 import org.apache.ratis.util.ProtoUtils;
@@ -39,7 +39,7 @@ import java.util.Queue;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 
-import static org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto.RaftNettyServerReplyCase.EXCEPTIONREPLY;
+import static org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerReplyProto.RaftNettyServerReplyCase.EXCEPTIONREPLY;
 
 public class NettyRpcProxy implements Closeable {
   public static class PeerMap extends PeerProxyMap<NettyRpcProxy> {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java
index 3bc2608..b763d58 100644
--- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java
@@ -21,12 +21,12 @@ import org.apache.ratis.client.impl.ClientProtoUtils;
 import org.apache.ratis.client.impl.RaftClientRpcWithProxy;
 import org.apache.ratis.netty.NettyRpcProxy;
 import org.apache.ratis.protocol.*;
-import org.apache.ratis.shaded.proto.RaftProtos;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.GroupManagementRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
-import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.proto.RaftProtos.RaftRpcRequestProto;
+import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto;
+import org.apache.ratis.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
index f6fcbc6..adcd786 100644
--- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
@@ -27,21 +27,21 @@ import org.apache.ratis.rpc.SupportedRpcType;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerRpc;
 import org.apache.ratis.server.impl.RaftServerRpcWithProxy;
-import org.apache.ratis.shaded.io.netty.bootstrap.ServerBootstrap;
-import org.apache.ratis.shaded.io.netty.channel.*;
-import org.apache.ratis.shaded.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.ratis.shaded.io.netty.channel.socket.SocketChannel;
-import org.apache.ratis.shaded.io.netty.channel.socket.nio.NioServerSocketChannel;
-import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
-import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufEncoder;
-import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
-import org.apache.ratis.shaded.io.netty.handler.logging.LogLevel;
-import org.apache.ratis.shaded.io.netty.handler.logging.LoggingHandler;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
-import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyExceptionReplyProto;
-import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto;
-import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
+import org.apache.ratis.thirdparty.io.netty.bootstrap.ServerBootstrap;
+import org.apache.ratis.thirdparty.io.netty.channel.*;
+import org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.ratis.thirdparty.io.netty.channel.socket.SocketChannel;
+import org.apache.ratis.thirdparty.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.ratis.thirdparty.io.netty.handler.codec.protobuf.ProtobufDecoder;
+import org.apache.ratis.thirdparty.io.netty.handler.codec.protobuf.ProtobufEncoder;
+import org.apache.ratis.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import org.apache.ratis.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
+import org.apache.ratis.thirdparty.io.netty.handler.logging.LogLevel;
+import org.apache.ratis.thirdparty.io.netty.handler.logging.LoggingHandler;
+import org.apache.ratis.proto.RaftProtos.*;
+import org.apache.ratis.proto.netty.NettyProtos.RaftNettyExceptionReplyProto;
+import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerReplyProto;
+import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto;
 import org.apache.ratis.util.CodeInjectionForTesting;
 import org.apache.ratis.util.ProtoUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/pom.xml b/ratis-proto-shaded/pom.xml
deleted file mode 100644
index 87e3766..0000000
--- a/ratis-proto-shaded/pom.xml
+++ /dev/null
@@ -1,482 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>ratis</artifactId>
-    <groupId>org.apache.ratis</groupId>
-    <version>0.3.0-SNAPSHOT</version>
-  </parent>
-
-  <artifactId>ratis-proto-shaded</artifactId>
-  <name>Apache Ratis Proto Shaded</name>
-
-  <properties>
-    <maven.javadoc.skip>true</maven.javadoc.skip>
-    <!--The Default target dir-->
-    <classes.dir>${project.build.directory}/classes</classes.dir>
-
-    <!--Version of grpc to be shaded -->
-    <shaded.grpc.version>1.14.0</shaded.grpc.version>
-    <!--Version of Netty to be shaded -->
-    <shaded.netty.version>4.1.27.Final</shaded.netty.version>
-    <netty-tcnative.version>2.0.8.Final</netty-tcnative.version>
-  </properties>
-
-  <build>
-    <extensions>
-      <!-- Use os-maven-plugin to initialize the "os.detected" properties -->
-      <extension>
-        <groupId>kr.motd.maven</groupId>
-        <artifactId>os-maven-plugin</artifactId>
-        <version>1.5.0.Final</version>
-      </extension>
-    </extensions>
-    <!--I want to override these in profile so define them
-         with variables up here-->
-    <sourceDirectory>${shaded.sources.dir}</sourceDirectory>
-    <outputDirectory>${classes.dir}</outputDirectory>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <compilerArgs>
-            <!-- disable all javac warnings for shaded sources -->
-            <arg>-Xlint:none</arg>
-            <arg>-XDignore.symbol.file</arg>
-          </compilerArgs>
-          <showWarnings>false</showWarnings>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-checkstyle-plugin</artifactId>
-        <configuration>
-          <excludes>org/apache/ratis/shaded/**/*</excludes>
-        </configuration>
-      </plugin>
-      <!-- Make a jar and put the sources in the jar -->
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-source-plugin</artifactId>
-      </plugin>
-      <plugin>
-        <!--Make it so assembly:single does nothing in here-->
-        <artifactId>maven-assembly-plugin</artifactId>
-        <configuration>
-          <skipAssembly>true</skipAssembly>
-        </configuration>
-      </plugin>
-    </plugins>
-    <pluginManagement>
-      <plugins>
-        <!--This plugin's configuration is used to store Eclipse m2e settings only.
-          It has no influence on the Maven build itself. -->
-        <plugin>
-          <groupId>org.eclipse.m2e</groupId>
-          <artifactId>lifecycle-mapping</artifactId>
-          <version>1.0.0</version>
-          <configuration>
-            <lifecycleMappingMetadata>
-              <pluginExecutions>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-shade-plugin</artifactId>
-                    <versionRange>[3.1.1,)</versionRange>
-                    <goals>
-                      <goal>shade</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <ignore/>
-                  </action>
-                </pluginExecution>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-antrun-plugin</artifactId>
-                    <versionRange>[1.7,)</versionRange>
-                    <goals>
-                      <goal>run</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <ignore/>
-                  </action>
-                </pluginExecution>
-              </pluginExecutions>
-            </lifecycleMappingMetadata>
-          </configuration>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-  </build>
-
-  <dependencies>
-    <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-      <version>${shaded.protobuf.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.protobuf.nano</groupId>
-      <artifactId>protobuf-javanano</artifactId>
-      <version>${shaded.protobuf-javanano.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-netty</artifactId>
-      <version>${shaded.grpc.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-protobuf</artifactId>
-      <version>${shaded.grpc.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-stub</artifactId>
-      <version>${shaded.grpc.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-context</artifactId>
-      <version>${shaded.grpc.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.auto</groupId>
-      <artifactId>auto-common</artifactId>
-      <version>0.8</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.auto.service</groupId>
-      <artifactId>auto-service</artifactId>
-      <version>1.0-rc4</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.auto.value</groupId>
-      <artifactId>auto-value-annotations</artifactId>
-      <version>1.6</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.auto.value</groupId>
-      <artifactId>auto-value</artifactId>
-      <version>1.6</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-      <version>24.1-jre</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
-      <version>3.0.2</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.errorprone</groupId>
-      <artifactId>error_prone_annotations</artifactId>
-      <version>2.2.0</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.j2objc</groupId>
-      <artifactId>j2objc-annotations</artifactId>
-      <version>1.3</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.squareup</groupId>
-      <artifactId>javapoet</artifactId>
-      <version>1.10.0</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.codehaus.mojo</groupId>
-      <artifactId>animal-sniffer-annotations</artifactId>
-      <version>1.16</version>
-    </dependency>
-
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-all</artifactId>
-      <version>${shaded.netty.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-tcnative</artifactId>
-      <version>${netty-tcnative.version}</version>
-      <classifier>${os.detected.classifier}</classifier>
-    </dependency>
-
-    <dependency>
-      <groupId>org.jboss.marshalling</groupId>
-      <artifactId>jboss-marshalling</artifactId>
-      <version>1.4.11.Final</version>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-      <version>1.2</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-api</artifactId>
-      <version>2.11.0</version>
-    </dependency>
-    <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
-      <version>1.2.17</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.jcraft</groupId>
-      <artifactId>jzlib</artifactId>
-      <version>1.1.3</version>
-    </dependency>
-    <dependency>
-      <groupId>com.ning</groupId>
-      <artifactId>compress-lzf</artifactId>
-      <version>1.0.3</version>
-    </dependency>
-    <dependency>
-      <groupId>net.jpountz.lz4</groupId>
-      <artifactId>lz4</artifactId>
-      <version>1.3.0</version>
-    </dependency>
-    <dependency>
-      <groupId>org.bouncycastle</groupId>
-      <artifactId>bcpkix-jdk15on</artifactId>
-      <version>1.54</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.code.gson</groupId>
-      <artifactId>gson</artifactId>
-      <version>2.8.2</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.eclipse.jetty.npn</groupId>
-      <artifactId>npn-api</artifactId>
-      <version>1.1.1.v20141010</version>
-      <scope>provided</scope> <!-- Provided by npn-boot -->
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty.alpn</groupId>
-      <artifactId>alpn-api</artifactId>
-      <version>1.1.2.v20150522</version>
-      <scope>provided</scope> <!-- Provided by alpn-boot -->
-    </dependency>
-
-    <dependency>
-      <groupId>org.jctools</groupId>
-      <artifactId>jctools-core</artifactId>
-      <version>2.1.2</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.fasterxml</groupId>
-      <artifactId>aalto-xml</artifactId>
-      <version>1.0.0</version>
-    </dependency>
-
-    <dependency>
-      <groupId>io.opencensus</groupId>
-      <artifactId>opencensus-api</artifactId>
-      <version>${io.opencensus.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>io.opencensus</groupId>
-      <artifactId>opencensus-contrib-grpc-metrics</artifactId>
-      <version>${io.opencensus.version}</version>
-    </dependency>
-  </dependencies>
-
-  <profiles>
-    <profile>
-      <id>compile-protobuf</id>
-      <!--
-         Shade and drop the generated java files under target/shaded-sources.
-         Run this profile/step everytime you change proto
-         files or update the protobuf version.
-      -->
-      <activation>
-        <property>
-          <name>!skipShade</name>
-        </property>
-      </activation>
-      <properties>
-      </properties>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.xolstice.maven.plugins</groupId>
-            <artifactId>protobuf-maven-plugin</artifactId>
-            <configuration>
-              <protocArtifact>
-                com.google.protobuf:protoc:${shaded.protobuf.version}:exe:${os.detected.classifier}
-              </protocArtifact>
-              <!-- Place these in a location that compiler-plugin is already looking -->
-              <outputDirectory>${project.build.directory}/generated-sources</outputDirectory>
-              <!-- With multiple executions, this must be `false` otherwise we wipe out the previous execution -->
-              <clearOutputDirectory>false</clearOutputDirectory>
-            </configuration>
-            <executions>
-              <execution>
-                <id>1</id>
-                <phase>generate-sources</phase>
-                <goals>
-                  <goal>compile</goal>
-                </goals>
-              </execution>
-              <execution>
-                <id>2</id>
-                <phase>generate-sources</phase>
-                <goals>
-                  <goal>compile-custom</goal>
-                </goals>
-                <configuration>
-                  <pluginId>grpc-java</pluginId>
-                  <pluginArtifact>
-                    io.grpc:protoc-gen-grpc-java:${shaded.grpc.version}:exe:${os.detected.classifier}
-                  </pluginArtifact>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <artifactId>maven-jar-plugin</artifactId>
-            <executions>
-              <execution>
-                <!-- Replace the "default" jar-plugin execution -->
-                <!-- This is a super-dirty hack to work around Yetus
-                     PreCommit not using the package lifecycle phase -->
-                <id>default-jar</id>
-                <phase>process-classes</phase>
-                <goals>
-                  <goal>jar</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-shade-plugin</artifactId>
-            <executions>
-              <execution>
-                <phase>process-classes</phase>
-                <goals>
-                  <goal>shade</goal>
-                </goals>
-                <configuration>
-                  <shadeSourcesContent>true</shadeSourcesContent>
-                  <createSourcesJar>true</createSourcesJar>
-                  <!-- Replace the original artifact which is no good on its own -->
-                  <shadedArtifactAttached>false</shadedArtifactAttached>
-                  <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml</dependencyReducedPomLocation>
-                  <relocations>
-                    <relocation>
-                      <pattern>com.google.api</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.api</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>com.google.cloud.audit</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.cloud.audit</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>com.google.common</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.common</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>com.google.logging.type</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.logging.type</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>com.google.longrunning</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.longrunning</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>com.google.protobuf</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.protobuf</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>com.google.rpc</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.rpc</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>com.google.thirdparty.publicsuffix</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.thirdparty.publicsuffix</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>com.google.type</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.type</shadedPattern>
-                    </relocation>
-
-                    <relocation>
-                      <pattern>io.grpc</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.io.grpc</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>io.netty</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.io.netty</shadedPattern>
-                    </relocation>
-
-                    <relocation>
-                      <pattern>io.opencensus</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.io.opencensus</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>com.google.gson</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.gson</shadedPattern>
-                    </relocation>
-                  </relocations>
-
-                  <artifactSet>
-                    <includes>
-                      <include>com.google.api.grpc:proto-google-common-protos</include>
-                      <include>com.google.code.gson:gson</include>
-                      <include>com.google.guava:guava</include>
-                      <include>com.google.protobuf.nano:protobuf-javanano</include>
-                      <include>com.google.protobuf:protobuf-java-util</include>
-                      <include>com.google.protobuf:protobuf-java</include>
-                      <include>io.grpc:grpc-context</include>
-                      <include>io.grpc:grpc-core</include>
-                      <include>io.grpc:grpc-netty</include>
-                      <include>io.grpc:grpc-protobuf-lite</include>
-                      <include>io.grpc:grpc-protobuf</include>
-                      <include>io.grpc:grpc-stub</include>
-                      <include>io.netty:netty-all</include>
-                      <include>io.opencensus:opencensus-api</include>
-                      <include>io.opencensus:opencensus-contrib-grpc-metrics</include>
-                    </includes>
-                  </artifactSet>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/proto/Examples.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/Examples.proto b/ratis-proto-shaded/src/main/proto/Examples.proto
deleted file mode 100644
index 6efef5b..0000000
--- a/ratis-proto-shaded/src/main/proto/Examples.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 = "proto3";
-option java_package = "org.apache.ratis.shaded.proto";
-option java_outer_classname = "ExamplesProtos";
-option java_generate_equals_and_hash = true;
-package ratis.example;
-
-message FileStoreRequestProto {
-  oneof Request {
-    WriteRequestHeaderProto writeHeader = 1;
-    WriteRequestProto write = 2;
-    DeleteRequestProto delete = 3;
-  }
-}
-
-message ReadRequestProto {
-  bytes path = 1;
-  uint64 offset = 2;
-  uint64 length = 3;
-}
-
-message WriteRequestHeaderProto {
-  bytes path = 1;
-  bool close = 2; // close the file after write?
-  uint64 offset = 3;
-}
-
-message WriteRequestProto {
-  WriteRequestHeaderProto header = 1;
-  bytes data = 2;
-}
-
-message DeleteRequestProto {
-  bytes path = 1;
-}
-
-message ReadReplyProto {
-  bytes resolvedPath = 1;
-  uint64 offset = 2;
-  bytes data = 3; // returned data size may be smaller than the requested size
-}
-
-message WriteReplyProto {
-  bytes resolvedPath = 1;
-  uint64 offset = 2;
-  uint64 length = 3; // bytes actually written
-}
-
-message DeleteReplyProto {
-  bytes resolvedPath = 1;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/proto/Grpc.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/Grpc.proto b/ratis-proto-shaded/src/main/proto/Grpc.proto
deleted file mode 100644
index 5c4bbad..0000000
--- a/ratis-proto-shaded/src/main/proto/Grpc.proto
+++ /dev/null
@@ -1,54 +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 = "proto3";
-option java_package = "org.apache.ratis.shaded.proto.grpc";
-option java_outer_classname = "GrpcProtos";
-option java_generate_equals_and_hash = true;
-package ratis.grpc;
-
-import "Raft.proto";
-
-service RaftClientProtocolService {
-  // A client-to-server RPC to set new raft configuration
-  rpc setConfiguration(ratis.common.SetConfigurationRequestProto)
-      returns(ratis.common.RaftClientReplyProto) {}
-
-  // A client-to-server stream RPC to append data
-  rpc append(stream ratis.common.RaftClientRequestProto)
-      returns (stream ratis.common.RaftClientReplyProto) {}
-}
-
-service RaftServerProtocolService {
-  rpc requestVote(ratis.common.RequestVoteRequestProto)
-      returns(ratis.common.RequestVoteReplyProto) {}
-
-  rpc appendEntries(stream ratis.common.AppendEntriesRequestProto)
-      returns(stream ratis.common.AppendEntriesReplyProto) {}
-
-  rpc installSnapshot(stream ratis.common.InstallSnapshotRequestProto)
-      returns(ratis.common.InstallSnapshotReplyProto) {}
-}
-
-service AdminProtocolService {
-  // A client-to-server RPC to add a new group
-  rpc groupManagement(ratis.common.GroupManagementRequestProto)
-      returns(ratis.common.RaftClientReplyProto) {}
-
-  rpc serverInformation(ratis.common.ServerInformationRequestProto)
-      returns(ratis.common.ServerInformationReplyProto) {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/proto/Hadoop.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/Hadoop.proto b/ratis-proto-shaded/src/main/proto/Hadoop.proto
deleted file mode 100644
index 0d6107e..0000000
--- a/ratis-proto-shaded/src/main/proto/Hadoop.proto
+++ /dev/null
@@ -1,51 +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 = "proto3";
-option java_package = "org.apache.ratis.shaded.proto.hadoop";
-option java_outer_classname = "HadoopProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-package ratis.hadoop;
-
-import "Raft.proto";
-
-service CombinedClientProtocolService {
-  rpc submitClientRequest(ratis.common.RaftClientRequestProto)
-      returns(ratis.common.RaftClientReplyProto);
-
-  rpc setConfiguration(ratis.common.SetConfigurationRequestProto)
-      returns(ratis.common.RaftClientReplyProto);
-
-  rpc groupManagement(ratis.common.GroupManagementRequestProto)
-      returns(ratis.common.RaftClientReplyProto);
-
-  rpc serverInformation(ratis.common.ServerInformationRequestProto)
-      returns(ratis.common.ServerInformationReplyProto);
-}
-
-service RaftServerProtocolService {
-  rpc requestVote(ratis.common.RequestVoteRequestProto)
-      returns(ratis.common.RequestVoteReplyProto);
-
-  rpc appendEntries(ratis.common.AppendEntriesRequestProto)
-      returns(ratis.common.AppendEntriesReplyProto);
-
-  rpc installSnapshot(ratis.common.InstallSnapshotRequestProto)
-      returns(ratis.common.InstallSnapshotReplyProto);
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/proto/Logservice.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/Logservice.proto b/ratis-proto-shaded/src/main/proto/Logservice.proto
deleted file mode 100644
index fd586bf..0000000
--- a/ratis-proto-shaded/src/main/proto/Logservice.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 = "proto3";
-option java_package = "org.apache.ratis.shaded.proto.logservice";
-option java_outer_classname = "LogServiceProtos";
-option java_generate_equals_and_hash = true;
-package ratis.logservice;
-
-enum MessageType {
-	READ_REQUEST = 0;
-	READ_REPLY = 1;
-	WRITE = 2;
-}
-
-message LogMessage {
-	MessageType type = 1;
-	string log_name = 2;
-	uint64 length = 3;
-	bytes  data = 4;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/proto/Netty.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/Netty.proto b/ratis-proto-shaded/src/main/proto/Netty.proto
deleted file mode 100644
index 40aa498..0000000
--- a/ratis-proto-shaded/src/main/proto/Netty.proto
+++ /dev/null
@@ -1,52 +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 = "proto3";
-option java_package = "org.apache.ratis.shaded.proto.netty";
-option java_outer_classname = "NettyProtos";
-option java_generate_equals_and_hash = true;
-package ratis.netty;
-
-import "Raft.proto";
-
-message RaftNettyExceptionReplyProto {
-  ratis.common.RaftRpcReplyProto rpcReply = 1;
-  bytes exception = 2;
-}
-
-message RaftNettyServerRequestProto {
-  oneof raftNettyServerRequest {
-    ratis.common.RequestVoteRequestProto requestVoteRequest = 1;
-    ratis.common.AppendEntriesRequestProto appendEntriesRequest = 2;
-    ratis.common.InstallSnapshotRequestProto installSnapshotRequest = 3;
-    ratis.common.RaftClientRequestProto raftClientRequest = 4;
-    ratis.common.SetConfigurationRequestProto setConfigurationRequest = 5;
-    ratis.common.GroupManagementRequestProto groupManagementRequest = 6;
-    ratis.common.ServerInformationRequestProto serverInformationRequest = 7;
-  }
-}
-
-message RaftNettyServerReplyProto {
-  oneof raftNettyServerReply {
-    ratis.common.RequestVoteReplyProto requestVoteReply = 1;
-    ratis.common.AppendEntriesReplyProto appendEntriesReply = 2;
-    ratis.common.InstallSnapshotReplyProto installSnapshotReply = 3;
-    ratis.common.RaftClientReplyProto raftClientReply = 4;
-    ratis.common.ServerInformationReplyProto serverInfoReply = 5;
-    RaftNettyExceptionReplyProto exceptionReply = 6;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/proto/RMap.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/RMap.proto b/ratis-proto-shaded/src/main/proto/RMap.proto
deleted file mode 100644
index 33ce4fb..0000000
--- a/ratis-proto-shaded/src/main/proto/RMap.proto
+++ /dev/null
@@ -1,189 +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 = "proto3";
-option java_package = "org.apache.ratis.shaded.proto.rmap";
-option java_outer_classname = "RMapProtos";
-option java_generate_equals_and_hash = true;
-package ratis.rmap;
-
-// TODO: This .proto file should go to the ratis-replicated-map module, but we need it here
-// due to shading.
-
-// Metadata about a replicated map
-message RMapInfo {
-    int64 rmap_id = 1;
-    string name = 2;
-    string key_class = 3;
-    string value_class = 4;
-    string key_serde_class = 5;
-    string value_serde_class = 6;
-    string key_comparator_class = 7;
-}
-
-// An entry in a replicated map.`
-message Entry {
-    bytes key = 1;
-    bytes value = 2;
-}
-
-// TODO: raft client should allow a Service decleration, and calling a method from the service
-// similar to how coprocessor calls work in HBase.
-message Request {
-    oneof RequestType {
-        MultiActionRequest multi_action_request = 1;
-        ScanRequest scan_request = 2;
-        CreateRMapRequest create_rmap_request = 3;
-        DeleteRMapRequest delete_rmap_request = 4;
-        ListRMapInfosRequest list_rmap_infos_request = 5;
-    }
-}
-
-message Response {
-    ExceptionResponse exception = 1;
-    oneof ResponseType {
-        MultiActionResponse multi_action_response = 2;
-        ScanResponse scan_response = 3;
-        CreateRMapResponse create_rmap_response = 4;
-        DeleteRMapResponse delete_rmap_response = 5;
-        ListRMapInfosResponse list_rmap_infos_response = 6;
-    }
-}
-
-message MultiActionRequest {
-    int64 rmap_id = 1;
-    repeated Action action = 2;
-}
-
-message Action {
-    oneof ActionType {
-        GetRequest get_request = 1;
-        PutRequest put_request = 2;
-    }
-}
-
-message ActionResponse {
-    oneof ActionType {
-        GetResponse get_response = 1;
-        PutResponse put_response = 2;
-    }
-}
-
-message MultiActionResponse {
-    repeated ActionResponse action_response = 1;
-}
-
-message CreateRMapRequest {
-    RMapInfo rmap_info = 1;
-}
-
-message CreateRMapResponse {
-    RMapInfo rmap_info = 1;
-}
-
-message DeleteRMapRequest {
-    int64 rmap_id = 1;
-}
-
-message DeleteRMapResponse {
-}
-
-message ListRMapInfosRequest {
-    oneof ListRMapInfosType {
-        // if set, we only care about a particular RMapInfo
-        int64 rmap_id = 1;
-
-        // only return infos whose names match this pattern
-        string name_pattern = 2;
-    }
-}
-
-message ListRMapInfosResponse {
-    repeated RMapInfo rmap_info = 1;
-}
-
-message GetRequest {
-    bytes key = 1;
-}
-
-message GetResponse {
-    bool found = 1;
-    bytes key = 2;
-    bytes value = 3;
-}
-
-message PutRequest {
-    bytes key = 1;
-    bytes value = 2;
-}
-
-message PutResponse {
-}
-
-message Scan {
-    bytes start_Key = 1;
-    bytes end_key = 2;
-    bool start_key_inclusive = 3;
-    bool end_key_inclusive = 4;
-    bool keys_only = 5;
-    int32 limit = 6;
-}
-
-message ScanRequest {
-    int64 rmap_id = 1;
-    Scan scan = 2;
-}
-
-message ScanResponse {
-    repeated Entry entry = 1;
-}
-
-message ExceptionResponse {
-    // Class name of the exception thrown from the server
-    string exception_class_name = 1;
-    // Exception stack trace from the server side
-    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.
-    string hostname = 3;
-    int32 port = 4;
-    // Set if we are NOT to retry on receipt of this exception
-    bool do_not_retry = 5;
-}
-
-message Id {
-    int64 id = 1;
-}
-
-message WALEntry {
-    int64 rmap_id = 1;  // these are shared by all WALEntry types
-    repeated Entry entry = 2;
-    oneof WALEntryType {
-        // Multi is not here, because we do not want to create one more object unnecessarily
-        CreateRMapWALEntry create_rmap_entry = 3;
-        DeleteRMapWALEntry delete_rmap_entry = 4;
-    }
-}
-
-message CreateRMapWALEntry {
-    RMapInfo rmap_info = 1;
-    Id id = 2;
-}
-
-message DeleteRMapWALEntry {
-    int64 id = 1;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/proto/Raft.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/Raft.proto b/ratis-proto-shaded/src/main/proto/Raft.proto
deleted file mode 100644
index 0a93e95..0000000
--- a/ratis-proto-shaded/src/main/proto/Raft.proto
+++ /dev/null
@@ -1,307 +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 = "proto3";
-option java_package = "org.apache.ratis.shaded.proto";
-option java_outer_classname = "RaftProtos";
-option java_generate_equals_and_hash = true;
-package ratis.common;
-
-message RaftPeerProto {
-  bytes id = 1;      // id of the peer
-  string address = 2; // e.g. IP address, hostname etc.
-}
-
-message RaftGroupIdProto {
-  bytes id = 1;
-}
-
-message RaftGroupProto {
-  RaftGroupIdProto groupId = 1;
-  repeated RaftPeerProto peers = 2;
-}
-
-message RaftConfigurationProto {
-  repeated RaftPeerProto peers = 1; // the peers in the current or new conf
-  repeated RaftPeerProto oldPeers = 2; // the peers in the old conf
-}
-
-message SMLogEntryProto {
-  // TODO: This is not super efficient if the SM itself uses PB to serialize its own data for a
-  // log entry. Data will be copied twice. We should directly support having any Message from SM
-  bytes data = 1;
-
-  bytes stateMachineData = 2; // State machine specific data which is not written to log.
-  bool stateMachineDataAttached = 3; // set this flag when state machine data is attached.
-  uint64 serializedProtobufSize = 4; // size of the serialized LogEntryProto along with stateMachineData
-}
-
-message LeaderNoOp {
-  // empty
-}
-
-message LogEntryProto {
-  uint64 term = 1;
-  uint64 index = 2;
-
-  oneof LogEntryBody {
-    SMLogEntryProto smLogEntry = 3;
-    RaftConfigurationProto configurationEntry = 4;
-    LeaderNoOp noOp = 5;
-  }
-
-  // clientId and callId are used to rebuild the retry cache. They're not
-  // necessary for configuration change since re-conf is idempotent.
-  bytes clientId = 6;
-  uint64 callId = 7;
-}
-
-message TermIndexProto {
-  uint64 term = 1;
-  uint64 index = 2;
-}
-
-message RaftRpcRequestProto {
-  bytes requestorId = 1;
-  bytes replyId = 2;
-  RaftGroupIdProto raftGroupId = 3;
-  uint64 callId = 4;
-
-  uint64 seqNum = 15;
-}
-
-message RaftRpcReplyProto {
-  bytes requestorId = 1;
-  bytes replyId = 2;
-  RaftGroupIdProto raftGroupId = 3;
-  uint64 callId = 4;
-
-  bool success = 15;
-}
-
-message FileChunkProto {
-  string filename = 1; // relative to root
-  uint64 totalSize = 2;
-  bytes fileDigest = 3;
-  uint32 chunkIndex = 4;
-  uint64 offset = 5;
-  bytes data = 6;
-  bool done = 7;
-}
-
-enum InstallSnapshotResult {
-  SUCCESS = 0;
-  NOT_LEADER = 1;
-}
-
-message RequestVoteRequestProto {
-  RaftRpcRequestProto serverRequest = 1;
-  uint64 candidateTerm = 2;
-  TermIndexProto candidateLastEntry = 3;
-}
-
-message RequestVoteReplyProto {
-  RaftRpcReplyProto serverReply = 1;
-  uint64 term = 2;
-  bool shouldShutdown = 3;
-}
-
-message CommitInfoProto {
-  RaftPeerProto server = 1;
-  uint64 commitIndex = 2;
-}
-
-message AppendEntriesRequestProto {
-  RaftRpcRequestProto serverRequest = 1;
-  uint64 leaderTerm = 2;
-  TermIndexProto previousLog = 3;
-  repeated LogEntryProto entries = 4;
-  uint64 leaderCommit = 5;
-  bool initializing = 6;
-
-  repeated CommitInfoProto commitInfos = 15;
-}
-
-message AppendEntriesReplyProto {
-  enum AppendResult {
-    SUCCESS = 0;
-    NOT_LEADER = 1; // the requester's term is not large enough
-    INCONSISTENCY = 2; // gap between the local log and the entries
-  }
-
-  RaftRpcReplyProto serverReply = 1;
-  uint64 term = 2;
-  uint64 nextIndex = 3;
-  AppendResult result = 4;
-}
-
-message InstallSnapshotRequestProto {
-  RaftRpcRequestProto serverRequest = 1;
-  string requestId = 2; // an identifier for chunked-requests.
-  uint32 requestIndex = 3; // the index for this request chunk. Starts from 0.
-  RaftConfigurationProto raftConfiguration = 4;
-  uint64 leaderTerm = 5;
-  TermIndexProto termIndex = 6;
-  repeated FileChunkProto fileChunks = 7;
-  uint64 totalSize = 8;
-  bool done = 9; // whether this is the final chunk for the same req.
-}
-
-message InstallSnapshotReplyProto {
-  RaftRpcReplyProto serverReply = 1;
-  uint32 requestIndex = 2;
-  uint64 term = 3;
-  InstallSnapshotResult result = 4;
-}
-
-message ClientMessageEntryProto {
-  bytes content = 1;
-}
-
-enum ReplicationLevel {
-  MAJORITY = 0;
-  ALL = 1;
-}
-
-
-/** Role of raft peer */
-enum RaftPeerRole {
-  LEADER = 0;
-  CANDIDATE = 1;
-  FOLLOWER = 2;
-}
-
-message WriteRequestTypeProto {
-  ReplicationLevel replication = 1;
-}
-
-message ReadRequestTypeProto {
-}
-
-message StaleReadRequestTypeProto {
-  uint64 minIndex = 1;
-}
-
-// normal client request
-message RaftClientRequestProto {
-  RaftRpcRequestProto rpcRequest = 1;
-  ClientMessageEntryProto message = 2;
-
-  oneof Type {
-    WriteRequestTypeProto write = 3;
-    ReadRequestTypeProto read = 4;
-    StaleReadRequestTypeProto staleRead = 5;
-  }
-}
-
-message NotLeaderExceptionProto {
-  RaftPeerProto suggestedLeader = 1;
-  repeated RaftPeerProto peersInConf = 2;
-}
-
-message NotReplicatedExceptionProto {
-  uint64 callId = 1;
-  ReplicationLevel replication = 2;
-  uint64 logIndex = 3;
-}
-
-message StateMachineExceptionProto {
-  string exceptionClassName = 1;
-  string errorMsg = 2;
-  bytes stacktrace = 3;
-}
-
-message RaftClientReplyProto {
-  RaftRpcReplyProto rpcReply = 1;
-  ClientMessageEntryProto message = 2;
-
-  oneof ExceptionDetails {
-    NotLeaderExceptionProto notLeaderException = 3;
-    NotReplicatedExceptionProto notReplicatedException = 4;
-    StateMachineExceptionProto stateMachineException = 5;
-  }
-
-  repeated CommitInfoProto commitInfos = 15;
-}
-
-// setConfiguration request
-message SetConfigurationRequestProto {
-  RaftRpcRequestProto rpcRequest = 1;
-  repeated RaftPeerProto peers = 2;
-}
-
-// A request to add a new group
-message GroupAddRequestProto {
-  RaftGroupProto group = 1; // the group to be added.
-}
-
-message GroupRemoveRequestProto {
-  RaftGroupIdProto groupId = 1; // the group to be removed.
-  bool deleteDirectory = 2; // delete the directory for that group?
-}
-
-message GroupManagementRequestProto {
-  RaftRpcRequestProto rpcRequest = 1;
-
-  oneof Op {
-    GroupAddRequestProto groupAdd = 2;
-    GroupRemoveRequestProto groupRemove = 3;
-  }
-}
-
-// server info request
-message ServerInformationRequestProto {
-  RaftRpcRequestProto rpcRequest = 1;
-}
-
-message ServerRpcProto {
-  RaftPeerProto id = 1;
-  uint64 lastRpcElapsedTimeMs = 2;
-}
-
-message LeaderInfoProto {
-  repeated ServerRpcProto followerInfo = 1;
-}
-
-message FollowerInfoProto {
-  ServerRpcProto leaderInfo = 1;
-  bool inLogSync = 2;
-}
-
-message CandidateInfoProto {
-  uint64 lastLeaderElapsedTimeMs = 1;
-}
-
-message RoleInfoProto {
-  RaftPeerProto self = 1;
-  RaftPeerRole role = 2;
-  uint64 roleElapsedTimeMs = 3;
-
-  oneof PeerInfo {
-    LeaderInfoProto leaderInfo = 4;
-    FollowerInfoProto followerInfo = 5;
-    CandidateInfoProto candidateInfo = 6;
-  }
-}
-
-message ServerInformationReplyProto {
-  RaftRpcReplyProto rpcReply = 1;
-  RaftGroupProto group = 2;
-  RoleInfoProto role = 3;
-  bool isRaftStorageHealthy = 4;
-  repeated CommitInfoProto commitInfos = 5;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ManagedChannelProvider
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ManagedChannelProvider b/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ManagedChannelProvider
deleted file mode 100644
index dbf2d84..0000000
--- a/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ManagedChannelProvider
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.ratis.shaded.io.grpc.netty.NettyChannelProvider

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.NameResolverProvider
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.NameResolverProvider b/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.NameResolverProvider
deleted file mode 100644
index 439b1d8..0000000
--- a/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.NameResolverProvider
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.ratis.shaded.io.grpc.internal.DnsNameResolverProvider

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ServerProvider
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ServerProvider b/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ServerProvider
deleted file mode 100644
index f251467..0000000
--- a/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ServerProvider
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.ratis.shaded.io.grpc.netty.NettyServerProvider

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml
new file mode 100644
index 0000000..ffbfe70
--- /dev/null
+++ b/ratis-proto/pom.xml
@@ -0,0 +1,191 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>0.3.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>ratis-proto</artifactId>
+  <name>Apache Ratis Protocols</name>
+
+  <properties>
+    <maven.javadoc.skip>true</maven.javadoc.skip>
+  </properties>
+
+  <build>
+    <extensions>
+      <!-- Use os-maven-plugin to initialize the "os.detected" properties -->
+      <extension>
+        <groupId>kr.motd.maven</groupId>
+        <artifactId>os-maven-plugin</artifactId>
+        <version>1.5.0.Final</version>
+      </extension>
+    </extensions>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <compilerArgs>
+            <!-- disable all javac warnings for shaded sources -->
+            <arg>-Xlint:none</arg>
+            <arg>-XDignore.symbol.file</arg>
+          </compilerArgs>
+          <showWarnings>false</showWarnings>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <configuration/>
+      </plugin>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.xolstice.maven.plugins</groupId>
+        <artifactId>protobuf-maven-plugin</artifactId>
+        <configuration>
+          <protocArtifact>
+            com.google.protobuf:protoc:${shaded.protobuf.version}:exe:${os.detected.classifier}
+          </protocArtifact>
+          <!-- Place these in a location that compiler-plugin is already looking -->
+          <outputDirectory>${project.build.directory}/generated-sources</outputDirectory>
+          <!-- With multiple executions, this must be `false` otherwise we wipe out the previous execution -->
+          <clearOutputDirectory>false</clearOutputDirectory>
+        </configuration>
+        <executions>
+          <execution>
+            <id>compile-protobuf</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>compile-grpc</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>compile-custom</goal>
+            </goals>
+            <configuration>
+              <pluginId>grpc-java</pluginId>
+              <pluginArtifact>
+                io.grpc:protoc-gen-grpc-java:${shaded.grpc.version}:exe:${os.detected.classifier}
+              </pluginArtifact>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <!-- Replace the "default" jar-plugin execution -->
+            <!-- This is a super-dirty hack to work around Yetus
+                 PreCommit not using the package lifecycle phase -->
+            <id>default-jar</id>
+            <phase>process-classes</phase>
+            <goals>
+              <goal>jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <!-- Modify the generated source to use our shaded protobuf -->
+      <plugin>
+        <groupId>com.google.code.maven-replacer-plugin</groupId>
+        <artifactId>replacer</artifactId>
+        <version>1.5.3</version>
+        <executions>
+          <execution>
+            <phase>process-sources</phase>
+            <goals>
+              <goal>replace</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <basedir>${project.build.directory}/generated-sources</basedir>
+          <includes>
+              <include>**/*.java</include>
+          </includes>
+          <replacements>
+            <replacement>
+              <token>([^\.])com.google</token>
+              <value>$1org.apache.ratis.thirdparty.com.google</value>
+            </replacement>
+            <replacement>
+              <token>([^\.])io.grpc</token>
+              <value>$1org.apache.ratis.thirdparty.io.grpc</value>
+            </replacement>
+          </replacements>
+        </configuration>
+      </plugin>
+    </plugins>
+    <pluginManagement>
+      <plugins>
+        <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <version>1.0.0</version>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>
+                      com.google.code.maven-replacer-plugin
+                    </groupId>
+                    <artifactId>replacer</artifactId>
+                    <versionRange>
+                      [1.5.3,)
+                    </versionRange>
+                    <goals>
+                      <goal>replace</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore></ignore>
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.ratis</groupId>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto/src/main/proto/Examples.proto
----------------------------------------------------------------------
diff --git a/ratis-proto/src/main/proto/Examples.proto b/ratis-proto/src/main/proto/Examples.proto
new file mode 100644
index 0000000..c2e2500
--- /dev/null
+++ b/ratis-proto/src/main/proto/Examples.proto
@@ -0,0 +1,67 @@
+/**
+ * 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 = "proto3";
+option java_package = "org.apache.ratis.proto";
+option java_outer_classname = "ExamplesProtos";
+option java_generate_equals_and_hash = true;
+package ratis.example;
+
+message FileStoreRequestProto {
+  oneof Request {
+    WriteRequestHeaderProto writeHeader = 1;
+    WriteRequestProto write = 2;
+    DeleteRequestProto delete = 3;
+  }
+}
+
+message ReadRequestProto {
+  bytes path = 1;
+  uint64 offset = 2;
+  uint64 length = 3;
+}
+
+message WriteRequestHeaderProto {
+  bytes path = 1;
+  bool close = 2; // close the file after write?
+  uint64 offset = 3;
+}
+
+message WriteRequestProto {
+  WriteRequestHeaderProto header = 1;
+  bytes data = 2;
+}
+
+message DeleteRequestProto {
+  bytes path = 1;
+}
+
+message ReadReplyProto {
+  bytes resolvedPath = 1;
+  uint64 offset = 2;
+  bytes data = 3; // returned data size may be smaller than the requested size
+}
+
+message WriteReplyProto {
+  bytes resolvedPath = 1;
+  uint64 offset = 2;
+  uint64 length = 3; // bytes actually written
+}
+
+message DeleteReplyProto {
+  bytes resolvedPath = 1;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto/src/main/proto/Grpc.proto
----------------------------------------------------------------------
diff --git a/ratis-proto/src/main/proto/Grpc.proto b/ratis-proto/src/main/proto/Grpc.proto
new file mode 100644
index 0000000..3126fdb
--- /dev/null
+++ b/ratis-proto/src/main/proto/Grpc.proto
@@ -0,0 +1,54 @@
+/**
+ * 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 = "proto3";
+option java_package = "org.apache.ratis.proto.grpc";
+option java_outer_classname = "GrpcProtos";
+option java_generate_equals_and_hash = true;
+package ratis.grpc;
+
+import "Raft.proto";
+
+service RaftClientProtocolService {
+  // A client-to-server RPC to set new raft configuration
+  rpc setConfiguration(ratis.common.SetConfigurationRequestProto)
+      returns(ratis.common.RaftClientReplyProto) {}
+
+  // A client-to-server stream RPC to append data
+  rpc append(stream ratis.common.RaftClientRequestProto)
+      returns (stream ratis.common.RaftClientReplyProto) {}
+}
+
+service RaftServerProtocolService {
+  rpc requestVote(ratis.common.RequestVoteRequestProto)
+      returns(ratis.common.RequestVoteReplyProto) {}
+
+  rpc appendEntries(stream ratis.common.AppendEntriesRequestProto)
+      returns(stream ratis.common.AppendEntriesReplyProto) {}
+
+  rpc installSnapshot(stream ratis.common.InstallSnapshotRequestProto)
+      returns(ratis.common.InstallSnapshotReplyProto) {}
+}
+
+service AdminProtocolService {
+  // A client-to-server RPC to add a new group
+  rpc groupManagement(ratis.common.GroupManagementRequestProto)
+      returns(ratis.common.RaftClientReplyProto) {}
+
+  rpc serverInformation(ratis.common.ServerInformationRequestProto)
+      returns(ratis.common.ServerInformationReplyProto) {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto/src/main/proto/Hadoop.proto
----------------------------------------------------------------------
diff --git a/ratis-proto/src/main/proto/Hadoop.proto b/ratis-proto/src/main/proto/Hadoop.proto
new file mode 100644
index 0000000..dd220ef
--- /dev/null
+++ b/ratis-proto/src/main/proto/Hadoop.proto
@@ -0,0 +1,51 @@
+/**
+ * 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 = "proto3";
+option java_package = "org.apache.ratis.proto.hadoop";
+option java_outer_classname = "HadoopProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package ratis.hadoop;
+
+import "Raft.proto";
+
+service CombinedClientProtocolService {
+  rpc submitClientRequest(ratis.common.RaftClientRequestProto)
+      returns(ratis.common.RaftClientReplyProto);
+
+  rpc setConfiguration(ratis.common.SetConfigurationRequestProto)
+      returns(ratis.common.RaftClientReplyProto);
+
+  rpc groupManagement(ratis.common.GroupManagementRequestProto)
+      returns(ratis.common.RaftClientReplyProto);
+
+  rpc serverInformation(ratis.common.ServerInformationRequestProto)
+      returns(ratis.common.ServerInformationReplyProto);
+}
+
+service RaftServerProtocolService {
+  rpc requestVote(ratis.common.RequestVoteRequestProto)
+      returns(ratis.common.RequestVoteReplyProto);
+
+  rpc appendEntries(ratis.common.AppendEntriesRequestProto)
+      returns(ratis.common.AppendEntriesReplyProto);
+
+  rpc installSnapshot(ratis.common.InstallSnapshotRequestProto)
+      returns(ratis.common.InstallSnapshotReplyProto);
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto/src/main/proto/Logservice.proto
----------------------------------------------------------------------
diff --git a/ratis-proto/src/main/proto/Logservice.proto b/ratis-proto/src/main/proto/Logservice.proto
new file mode 100644
index 0000000..8ea32f2
--- /dev/null
+++ b/ratis-proto/src/main/proto/Logservice.proto
@@ -0,0 +1,35 @@
+/**
+ * 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 = "proto3";
+option java_package = "org.apache.ratis.proto.logservice";
+option java_outer_classname = "LogServiceProtos";
+option java_generate_equals_and_hash = true;
+package ratis.logservice;
+
+enum MessageType {
+	READ_REQUEST = 0;
+	READ_REPLY = 1;
+	WRITE = 2;
+}
+
+message LogMessage {
+	MessageType type = 1;
+	string log_name = 2;
+	uint64 length = 3;
+	bytes  data = 4;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto/src/main/proto/Netty.proto
----------------------------------------------------------------------
diff --git a/ratis-proto/src/main/proto/Netty.proto b/ratis-proto/src/main/proto/Netty.proto
new file mode 100644
index 0000000..b817e8b
--- /dev/null
+++ b/ratis-proto/src/main/proto/Netty.proto
@@ -0,0 +1,52 @@
+/**
+ * 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 = "proto3";
+option java_package = "org.apache.ratis.proto.netty";
+option java_outer_classname = "NettyProtos";
+option java_generate_equals_and_hash = true;
+package ratis.netty;
+
+import "Raft.proto";
+
+message RaftNettyExceptionReplyProto {
+  ratis.common.RaftRpcReplyProto rpcReply = 1;
+  bytes exception = 2;
+}
+
+message RaftNettyServerRequestProto {
+  oneof raftNettyServerRequest {
+    ratis.common.RequestVoteRequestProto requestVoteRequest = 1;
+    ratis.common.AppendEntriesRequestProto appendEntriesRequest = 2;
+    ratis.common.InstallSnapshotRequestProto installSnapshotRequest = 3;
+    ratis.common.RaftClientRequestProto raftClientRequest = 4;
+    ratis.common.SetConfigurationRequestProto setConfigurationRequest = 5;
+    ratis.common.GroupManagementRequestProto groupManagementRequest = 6;
+    ratis.common.ServerInformationRequestProto serverInformationRequest = 7;
+  }
+}
+
+message RaftNettyServerReplyProto {
+  oneof raftNettyServerReply {
+    ratis.common.RequestVoteReplyProto requestVoteReply = 1;
+    ratis.common.AppendEntriesReplyProto appendEntriesReply = 2;
+    ratis.common.InstallSnapshotReplyProto installSnapshotReply = 3;
+    ratis.common.RaftClientReplyProto raftClientReply = 4;
+    ratis.common.ServerInformationReplyProto serverInfoReply = 5;
+    RaftNettyExceptionReplyProto exceptionReply = 6;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-proto/src/main/proto/RMap.proto
----------------------------------------------------------------------
diff --git a/ratis-proto/src/main/proto/RMap.proto b/ratis-proto/src/main/proto/RMap.proto
new file mode 100644
index 0000000..43c9377
--- /dev/null
+++ b/ratis-proto/src/main/proto/RMap.proto
@@ -0,0 +1,189 @@
+/**
+ * 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 = "proto3";
+option java_package = "org.apache.ratis.proto.rmap";
+option java_outer_classname = "RMapProtos";
+option java_generate_equals_and_hash = true;
+package ratis.rmap;
+
+// TODO: This .proto file should go to the ratis-replicated-map module, but we need it here
+// due to shading.
+
+// Metadata about a replicated map
+message RMapInfo {
+    int64 rmap_id = 1;
+    string name = 2;
+    string key_class = 3;
+    string value_class = 4;
+    string key_serde_class = 5;
+    string value_serde_class = 6;
+    string key_comparator_class = 7;
+}
+
+// An entry in a replicated map.`
+message Entry {
+    bytes key = 1;
+    bytes value = 2;
+}
+
+// TODO: raft client should allow a Service decleration, and calling a method from the service
+// similar to how coprocessor calls work in HBase.
+message Request {
+    oneof RequestType {
+        MultiActionRequest multi_action_request = 1;
+        ScanRequest scan_request = 2;
+        CreateRMapRequest create_rmap_request = 3;
+        DeleteRMapRequest delete_rmap_request = 4;
+        ListRMapInfosRequest list_rmap_infos_request = 5;
+    }
+}
+
+message Response {
+    ExceptionResponse exception = 1;
+    oneof ResponseType {
+        MultiActionResponse multi_action_response = 2;
+        ScanResponse scan_response = 3;
+        CreateRMapResponse create_rmap_response = 4;
+        DeleteRMapResponse delete_rmap_response = 5;
+        ListRMapInfosResponse list_rmap_infos_response = 6;
+    }
+}
+
+message MultiActionRequest {
+    int64 rmap_id = 1;
+    repeated Action action = 2;
+}
+
+message Action {
+    oneof ActionType {
+        GetRequest get_request = 1;
+        PutRequest put_request = 2;
+    }
+}
+
+message ActionResponse {
+    oneof ActionType {
+        GetResponse get_response = 1;
+        PutResponse put_response = 2;
+    }
+}
+
+message MultiActionResponse {
+    repeated ActionResponse action_response = 1;
+}
+
+message CreateRMapRequest {
+    RMapInfo rmap_info = 1;
+}
+
+message CreateRMapResponse {
+    RMapInfo rmap_info = 1;
+}
+
+message DeleteRMapRequest {
+    int64 rmap_id = 1;
+}
+
+message DeleteRMapResponse {
+}
+
+message ListRMapInfosRequest {
+    oneof ListRMapInfosType {
+        // if set, we only care about a particular RMapInfo
+        int64 rmap_id = 1;
+
+        // only return infos whose names match this pattern
+        string name_pattern = 2;
+    }
+}
+
+message ListRMapInfosResponse {
+    repeated RMapInfo rmap_info = 1;
+}
+
+message GetRequest {
+    bytes key = 1;
+}
+
+message GetResponse {
+    bool found = 1;
+    bytes key = 2;
+    bytes value = 3;
+}
+
+message PutRequest {
+    bytes key = 1;
+    bytes value = 2;
+}
+
+message PutResponse {
+}
+
+message Scan {
+    bytes start_Key = 1;
+    bytes end_key = 2;
+    bool start_key_inclusive = 3;
+    bool end_key_inclusive = 4;
+    bool keys_only = 5;
+    int32 limit = 6;
+}
+
+message ScanRequest {
+    int64 rmap_id = 1;
+    Scan scan = 2;
+}
+
+message ScanResponse {
+    repeated Entry entry = 1;
+}
+
+message ExceptionResponse {
+    // Class name of the exception thrown from the server
+    string exception_class_name = 1;
+    // Exception stack trace from the server side
+    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.
+    string hostname = 3;
+    int32 port = 4;
+    // Set if we are NOT to retry on receipt of this exception
+    bool do_not_retry = 5;
+}
+
+message Id {
+    int64 id = 1;
+}
+
+message WALEntry {
+    int64 rmap_id = 1;  // these are shared by all WALEntry types
+    repeated Entry entry = 2;
+    oneof WALEntryType {
+        // Multi is not here, because we do not want to create one more object unnecessarily
+        CreateRMapWALEntry create_rmap_entry = 3;
+        DeleteRMapWALEntry delete_rmap_entry = 4;
+    }
+}
+
+message CreateRMapWALEntry {
+    RMapInfo rmap_info = 1;
+    Id id = 2;
+}
+
+message DeleteRMapWALEntry {
+    int64 id = 1;
+}


[3/3] incubator-ratis git commit: RATIS-316. Use ratis-thirdparty and reorganize generated code

Posted by el...@apache.org.
RATIS-316. Use ratis-thirdparty and reorganize generated code

* Use ratis-thirdparty to build Ratis
* Rename ratis-shaded-proto to ratis-proto
* Undo skipShade because we don't have to shade anymore
* Update BUILDING.md


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/3b9d50de
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/3b9d50de
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/3b9d50de

Branch: refs/heads/master
Commit: 3b9d50deddcb2dc846b62ab63dd668f45c8b6536
Parents: 523fb63
Author: Josh Elser <el...@apache.org>
Authored: Sat Sep 8 08:42:43 2018 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Fri Sep 21 11:14:24 2018 -0400

----------------------------------------------------------------------
 BUILDING.md                                     |  65 +--
 pom.xml                                         |  30 +-
 ratis-assembly/pom.xml                          |   6 +-
 ratis-client/pom.xml                            |   7 +-
 .../org/apache/ratis/client/RaftClient.java     |   2 +-
 .../ratis/client/impl/ClientProtoUtils.java     |  10 +-
 .../ratis/client/impl/RaftClientImpl.java       |   4 +-
 ratis-common/pom.xml                            |   7 +-
 .../java/org/apache/ratis/conf/ConfUtils.java   |   2 +-
 .../org/apache/ratis/protocol/ClientId.java     |   2 +-
 .../java/org/apache/ratis/protocol/Message.java |   2 +-
 .../ratis/protocol/NotReplicatedException.java  |   2 +-
 .../apache/ratis/protocol/RaftClientReply.java  |   2 +-
 .../ratis/protocol/RaftClientRequest.java       |   4 +-
 .../org/apache/ratis/protocol/RaftGroupId.java  |   2 +-
 .../java/org/apache/ratis/protocol/RaftId.java  |   2 +-
 .../org/apache/ratis/protocol/RaftPeerId.java   |   2 +-
 .../ratis/protocol/ServerInformationReply.java  |   4 +-
 .../java/org/apache/ratis/util/JmxRegister.java |   2 +-
 .../java/org/apache/ratis/util/ProtoUtils.java  |   8 +-
 .../java/org/apache/ratis/util/StringUtils.java |   6 +-
 ratis-examples/pom.xml                          |   6 +-
 .../arithmetic/ArithmeticStateMachine.java      |   4 +-
 .../examples/arithmetic/AssignmentMessage.java  |   2 +-
 .../ratis/examples/arithmetic/cli/Assign.java   |   4 +-
 .../ratis/examples/arithmetic/cli/Client.java   |   2 +-
 .../ratis/examples/arithmetic/cli/Server.java   |   2 +-
 .../ratis/examples/filestore/FileInfo.java      |   2 +-
 .../ratis/examples/filestore/FileStore.java     |   4 +-
 .../examples/filestore/FileStoreClient.java     |   4 +-
 .../examples/filestore/FileStoreCommon.java     |   2 +-
 .../filestore/FileStoreStateMachine.java        |  10 +-
 .../examples/filestore/FileStoreBaseTest.java   |   4 +-
 ratis-grpc/pom.xml                              |   7 +-
 .../java/org/apache/ratis/grpc/GrpcUtil.java    |   8 +-
 .../grpc/client/GrpcClientProtocolClient.java   |  22 +-
 .../grpc/client/GrpcClientProtocolProxy.java    |   6 +-
 .../grpc/client/GrpcClientProtocolService.java  |  12 +-
 .../apache/ratis/grpc/client/GrpcClientRpc.java |  12 +-
 .../ratis/grpc/client/GrpcClientStreamer.java   |   8 +-
 .../grpc/server/GrpcAdminProtocolService.java   |  12 +-
 .../ratis/grpc/server/GrpcLogAppender.java      |  10 +-
 .../grpc/server/GrpcServerProtocolClient.java   |  14 +-
 .../grpc/server/GrpcServerProtocolService.java  |   6 +-
 .../apache/ratis/grpc/server/GrpcService.java   |   6 +-
 .../org/apache/ratis/grpc/TestRaftStream.java   |   2 +-
 .../ratis/grpc/TestRetryCacheWithGrpc.java      |   4 +-
 ratis-hadoop-shaded/pom.xml                     | 138 +++---
 ratis-hadoop/pom.xml                            |   4 +-
 .../hadoop/ipc/ProtobufRpcEngineShaded.java     |   4 +-
 ...nedClientProtocolClientSideTranslatorPB.java |   2 +-
 .../client/CombinedClientProtocolPB.java        |   2 +-
 ...nedClientProtocolServerSideTranslatorPB.java |  16 +-
 .../hadooprpc/server/HadoopRpcService.java      |  12 +-
 .../hadooprpc/server/RaftServerProtocolPB.java  |   2 +-
 ...aftServerProtocolServerSideTranslatorPB.java |  16 +-
 ratis-logservice/pom.xml                        |   2 +-
 .../apache/ratis/logservice/api/LogMessage.java |   6 +-
 .../ratis/logservice/api/LogStateMachine.java   |   6 +-
 .../ratis/logservice/api/TestLogMessage.java    |   4 +-
 ratis-netty/pom.xml                             |   7 +-
 .../org/apache/ratis/netty/NettyClient.java     |  18 +-
 .../org/apache/ratis/netty/NettyRpcProxy.java   |  22 +-
 .../ratis/netty/client/NettyClientRpc.java      |  12 +-
 .../ratis/netty/server/NettyRpcService.java     |  30 +-
 ratis-proto-shaded/pom.xml                      | 482 -------------------
 .../src/main/proto/Examples.proto               |  67 ---
 ratis-proto-shaded/src/main/proto/Grpc.proto    |  54 ---
 ratis-proto-shaded/src/main/proto/Hadoop.proto  |  51 --
 .../src/main/proto/Logservice.proto             |  35 --
 ratis-proto-shaded/src/main/proto/Netty.proto   |  52 --
 ratis-proto-shaded/src/main/proto/RMap.proto    | 189 --------
 ratis-proto-shaded/src/main/proto/Raft.proto    | 307 ------------
 ....ratis.shaded.io.grpc.ManagedChannelProvider |  16 -
 ...he.ratis.shaded.io.grpc.NameResolverProvider |  16 -
 ...g.apache.ratis.shaded.io.grpc.ServerProvider |  16 -
 ratis-proto/pom.xml                             | 191 ++++++++
 ratis-proto/src/main/proto/Examples.proto       |  67 +++
 ratis-proto/src/main/proto/Grpc.proto           |  54 +++
 ratis-proto/src/main/proto/Hadoop.proto         |  51 ++
 ratis-proto/src/main/proto/Logservice.proto     |  35 ++
 ratis-proto/src/main/proto/Netty.proto          |  52 ++
 ratis-proto/src/main/proto/RMap.proto           | 189 ++++++++
 ratis-proto/src/main/proto/Raft.proto           | 307 ++++++++++++
 ratis-replicated-map/pom.xml                    |   6 +-
 ratis-server/pom.xml                            |   6 +-
 .../ratis/server/impl/CommitInfoCache.java      |   2 +-
 .../ratis/server/impl/LeaderElection.java       |   4 +-
 .../apache/ratis/server/impl/LeaderState.java   |   6 +-
 .../apache/ratis/server/impl/LogAppender.java   |   4 +-
 .../ratis/server/impl/PendingRequest.java       |   2 +-
 .../ratis/server/impl/PendingRequests.java      |   4 +-
 .../ratis/server/impl/RaftServerImpl.java       |  10 +-
 .../ratis/server/impl/RaftServerProxy.java      |   2 +-
 .../apache/ratis/server/impl/RetryCache.java    |   6 +-
 .../org/apache/ratis/server/impl/RoleInfo.java  |   2 +-
 .../ratis/server/impl/ServerProtoUtils.java     |   6 +-
 .../apache/ratis/server/impl/ServerState.java   |   8 +-
 .../ratis/server/impl/StateMachineUpdater.java  |   2 +-
 .../RaftServerAsynchronousProtocol.java         |   4 +-
 .../server/protocol/RaftServerProtocol.java     |  12 +-
 .../ratis/server/storage/LogInputStream.java    |   2 +-
 .../ratis/server/storage/LogOutputStream.java   |   4 +-
 .../apache/ratis/server/storage/LogReader.java  |   6 +-
 .../apache/ratis/server/storage/LogSegment.java |   8 +-
 .../ratis/server/storage/MemoryRaftLog.java     |   2 +-
 .../apache/ratis/server/storage/RaftLog.java    |   2 +-
 .../ratis/server/storage/RaftLogCache.java      |   2 +-
 .../ratis/server/storage/RaftLogWorker.java     |   2 +-
 .../ratis/server/storage/SegmentedRaftLog.java  |   2 +-
 .../ratis/server/storage/SnapshotManager.java   |   4 +-
 .../apache/ratis/statemachine/StateMachine.java |   4 +-
 .../ratis/statemachine/TransactionContext.java  |   8 +-
 .../statemachine/impl/BaseStateMachine.java     |   2 +-
 .../impl/TransactionContextImpl.java            |   8 +-
 .../java/org/apache/ratis/MiniRaftCluster.java  |   2 +-
 .../java/org/apache/ratis/RaftAsyncTests.java   |  10 +-
 .../java/org/apache/ratis/RaftBasicTests.java   |   4 +-
 .../java/org/apache/ratis/RaftTestUtil.java     |   6 +-
 .../TestRaftServerLeaderElectionTimeout.java    |   2 +-
 .../ratis/TestRaftServerSlownessDetection.java  |   2 +-
 .../org/apache/ratis/protocol/TestRaftId.java   |   2 +-
 .../impl/RaftReconfigurationBaseTest.java       |   2 +-
 .../ratis/server/impl/RetryCacheTestUtil.java   |   2 +-
 .../server/impl/ServerInformationBaseTest.java  |   2 +-
 .../server/simulation/RaftServerReply.java      |   6 +-
 .../server/simulation/RaftServerRequest.java    |   6 +-
 .../server/simulation/SimulatedServerRpc.java   |   2 +-
 .../server/storage/RaftStorageTestUtils.java    |   2 +-
 .../ratis/server/storage/TestCacheEviction.java |   2 +-
 .../ratis/server/storage/TestRaftLogCache.java  |   2 +-
 .../server/storage/TestRaftLogReadWrite.java    |   4 +-
 .../server/storage/TestRaftLogSegment.java      |   4 +-
 .../server/storage/TestSegmentedRaftLog.java    |   2 +-
 .../statemachine/RaftSnapshotBaseTest.java      |   2 +-
 .../SimpleStateMachine4Testing.java             |   8 +-
 .../ratis/statemachine/TestStateMachine.java    |   2 +-
 137 files changed, 1380 insertions(+), 1718 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/BUILDING.md
----------------------------------------------------------------------
diff --git a/BUILDING.md b/BUILDING.md
index 56b6556..c9a806b 100644
--- a/BUILDING.md
+++ b/BUILDING.md
@@ -28,54 +28,29 @@ $ mvn clean package -DskipTests
 Note: subsequent builds could be faster with skiping shading/protobuf compile steps.
 See the next section for more info.
 
-# Shading
+# Thirdparty
 
-We shade protos, protobuf and other libraries such as Netty, gRPC, Guava and Hadoop
-so that applications using Ratis may use protobuf and other libraries with versions
-different from the versions used here.
+We centralize all bundled thirdparty dependencies in the ratis-thirdparty module. This
+module is not attached to the core Ratis build as it only needs to change when one
+of these dependencies are changed. All dependencies included in ratis-thirdparty
+must be relocated to a different package to ensure no downstream classpath pollution.
 
-_Note: RATIS-288 changes how the shaded artifacts are generated, removing them from the
-source tree. Developers with local checkouts prior to this change will need to manually
-remove the directories `ratis-proto-shaded/src/main/java` and
-`ratis-hadoop-shaded/src/main/java`._
+Ratis developers should rely on these relocated thirdparty classes.
 
-By default, protobuf compilation and shaded jar creation are executed for every build.
-
-For developers who wish to skip protobuf generation and shaded jar creation because they
-are aware that they have not been modified, they can be disabled with the `skipShade` property.
-```
-$ mvn package -DskipTests -DskipShade
-```
-
-When the `skipShade` property is given, Maven will inspect your local Maven repository for
-the most recent version of `ratis-proto-shaded` (or `ratis-hadoop-shaded`), reaching out to
-Maven central when you have no local copy. You may need to run a `mvn install` prior to
-attempting to use the `skipShade` property to ensure that you have a version of the artifact
-available for your use.
-```
-$ mvn install -DskipTests
-```
-
-For developers familiar with the `skipCleanShade` option, this is no longer necessary. Maven's
-local repository is acting as a cache instead of the current working copy of your repository.
-`mvn clean` can be used to safely clean all temporary build files, without impacting your
-use of the `skipShade` option.
-
-Unit tests can also be executed with the `skipShade` option:
-```
-$ mvn package -DskipShade
-```
+As a result of this thirdparty module, there is no need for `skipShade` options in the
+build which previously existed because the shaded artifacts that are generated each
+build are limited only to the code in Ratis itself.
 
 ## What packages are shaded?
 
-| Original packages                   | Shaded packages                                              |
-| ------------------------------------|--------------------------------------------------------------|
-| `com.google.common`                 | `org.apache.ratis.shaded.com.google.common`                  |
-| `com.google.protobuf`               | `org.apache.ratis.shaded.com.google.protobuf`                |
-| `com.google.thirdparty.publicsuffix`| `org.apache.ratis.shaded.com.google.thirdparty.publicsuffix` |
-| `io.grpc`                           | `org.apache.ratis.shaded.io.grpc`                            |
-| `io.netty`                          | `org.apache.ratis.shaded.io.netty`                           |
-| `org.apache.hadoop.ipc.protobuf`    | `org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf`     |
-
-The compiled protocol-buffer definitions in this `ratis-proto-shaded` are stored in the
-`org.apache.ratis.shaded.proto` Java package.
+| Original packages                   | Shaded packages                                                  |
+| ------------------------------------|------------------------------------------------------------------|
+| `com.google.common`                 | `org.apache.ratis.thirdparty.com.google.common`                  |
+| `com.google.protobuf`               | `org.apache.ratis.thirdparty.com.google.protobuf`                |
+| `com.google.thirdparty.publicsuffix`| `org.apache.ratis.thirdparty.com.google.thirdparty.publicsuffix` |
+| `io.grpc`                           | `org.apache.ratis.thirdparty.io.grpc`                            |
+| `io.netty`                          | `org.apache.ratis.thirdparty.io.netty`                           |
+| `org.apache.hadoop.ipc.protobuf`    | `org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf`         |
+
+All compiled protocol-buffer definitions in `ratis-shaded` are stored in the
+`org.apache.ratis.proto` Java package.

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 133b0ea..8fa43ab 100644
--- a/pom.xml
+++ b/pom.xml
@@ -41,12 +41,14 @@
   </licenses>
 
   <modules>
+    <module>ratis-proto</module>
     <module>ratis-common</module>
     <module>ratis-client</module>
     <module>ratis-server</module>
 
     <module>ratis-grpc</module>
     <module>ratis-netty</module>
+    <module>ratis-hadoop-shaded</module>
     <module>ratis-hadoop</module>
 
     <module>ratis-assembly</module>
@@ -172,11 +174,13 @@
     <java.min.version>${javac.version}</java.min.version>
     <maven.min.version>3.3.9</maven.min.version>
 
-    <!--Version of protobuf to be shaded -->
+    <!-- Contains all shaded thirdparty dependencies -->
+    <ratis.thirdparty.version>0.1.0-SNAPSHOT</ratis.thirdparty.version>
+
+    <!-- Need these for the protobuf compiler. *MUST* match what is in ratis-thirdparty -->
     <shaded.protobuf.version>3.5.0</shaded.protobuf.version>
-    <shaded.protobuf-javanano.version>3.1.0</shaded.protobuf-javanano.version>
+    <shaded.grpc.version>1.14.0</shaded.grpc.version>
 
-    <io.opencensus.version>0.12.2</io.opencensus.version>
 
     <!-- Test properties -->
     <maven.test.redirectTestOutputToFile>true</maven.test.redirectTestOutputToFile>
@@ -272,7 +276,7 @@
       </dependency>
 
       <dependency>
-        <artifactId>ratis-proto-shaded</artifactId>
+        <artifactId>ratis-proto</artifactId>
         <groupId>org.apache.ratis</groupId>
         <version>${project.version}</version>
       </dependency>
@@ -309,6 +313,12 @@
         <version>${project.version}</version>
       </dependency>
 
+      <dependency>
+        <groupId>org.apache.ratis</groupId>
+        <artifactId>ratis-thirdparty</artifactId>
+        <version>${ratis.thirdparty.version}</version>
+      </dependency>
+
       <!-- External dependencies -->
       <dependency>
         <groupId>org.slf4j</groupId>
@@ -804,17 +814,5 @@
         </plugins>
       </build>
     </profile>
-    <profile>
-      <id>skipShade</id>
-      <activation>
-        <property>
-          <name>!skipShade</name>
-        </property>
-      </activation>
-      <modules>
-        <module>ratis-proto-shaded</module>
-        <module>ratis-hadoop-shaded</module>
-      </modules>
-    </profile>
   </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml
index 126f1ab..c773793 100644
--- a/ratis-assembly/pom.xml
+++ b/ratis-assembly/pom.xml
@@ -135,7 +135,11 @@
   <dependencies>
     <!-- Intra-project dependencies -->
     <dependency>
-      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-proto</artifactId>
       <groupId>org.apache.ratis</groupId>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-client/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml
index b12cb10..3e988be 100644
--- a/ratis-client/pom.xml
+++ b/ratis-client/pom.xml
@@ -25,9 +25,12 @@
 
   <dependencies>
     <dependency>
-      <artifactId>ratis-proto-shaded</artifactId>
       <groupId>org.apache.ratis</groupId>
-      <scope>provided</scope>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-proto</artifactId>
+      <groupId>org.apache.ratis</groupId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java b/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
index 725b456..5af28bb 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
@@ -25,7 +25,7 @@ import org.apache.ratis.protocol.*;
 import org.apache.ratis.retry.RetryPolicies;
 import org.apache.ratis.retry.RetryPolicy;
 import org.apache.ratis.rpc.RpcType;
-import org.apache.ratis.shaded.proto.RaftProtos.ReplicationLevel;
+import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
index b5a6172..f2a1e90 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
@@ -18,16 +18,16 @@
 package org.apache.ratis.client.impl;
 
 import org.apache.ratis.protocol.*;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.proto.RaftProtos.*;
 import org.apache.ratis.util.ProtoUtils;
 import org.apache.ratis.util.ReflectionUtils;
 
 import java.util.Arrays;
 
-import static org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.NOTLEADEREXCEPTION;
-import static org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.NOTREPLICATEDEXCEPTION;
-import static org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.STATEMACHINEEXCEPTION;
+import static org.apache.ratis.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.NOTLEADEREXCEPTION;
+import static org.apache.ratis.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.NOTREPLICATEDEXCEPTION;
+import static org.apache.ratis.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.STATEMACHINEEXCEPTION;
 
 public interface ClientProtoUtils {
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
index a07e229..aa4e9c8 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
@@ -24,7 +24,7 @@ import org.apache.ratis.retry.RetryPolicies;
 import org.apache.ratis.retry.RetryPolicy;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.protocol.*;
-import org.apache.ratis.shaded.proto.RaftProtos.ReplicationLevel;
+import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
 import org.apache.ratis.util.*;
 
 import java.io.IOException;
@@ -39,7 +39,7 @@ import java.util.function.LongFunction;
 import java.util.function.Supplier;
 import java.util.stream.Stream;
 
-import static org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto.TypeCase.STALEREAD;
+import static org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase.STALEREAD;
 
 /** A client who sends requests to a raft service. */
 final class RaftClientImpl implements RaftClient {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml
index 3810074..2419ee8 100644
--- a/ratis-common/pom.xml
+++ b/ratis-common/pom.xml
@@ -26,8 +26,11 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.ratis</groupId>
-      <artifactId>ratis-proto-shaded</artifactId>
-      <scope>provided</scope>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.ratis</groupId>
+      <artifactId>ratis-proto</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
index 6e7ae6d..430dbf0 100644
--- a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
+++ b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.conf;
 
-import org.apache.ratis.shaded.com.google.common.base.Objects;
+import org.apache.ratis.thirdparty.com.google.common.base.Objects;
 import org.apache.ratis.util.CheckedBiConsumer;
 import org.apache.ratis.util.NetUtils;
 import org.apache.ratis.util.SizeInBytes;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java
index f6894df..c8a591a 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 
 import java.util.UUID;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java b/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java
index 9e8198e..6ab04ae 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.MemoizedSupplier;
 import org.apache.ratis.util.StringUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/protocol/NotReplicatedException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/NotReplicatedException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/NotReplicatedException.java
index 67bda34..0a85b8e 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/NotReplicatedException.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/NotReplicatedException.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.shaded.proto.RaftProtos.ReplicationLevel;
+import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
 
 public class NotReplicatedException extends RaftException {
   private final long callId;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientReply.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientReply.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientReply.java
index ba3cdc7..5175482 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientReply.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientReply.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.shaded.proto.RaftProtos.CommitInfoProto;
+import org.apache.ratis.proto.RaftProtos.CommitInfoProto;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
index 34c96f3..a9fe740 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
@@ -17,12 +17,12 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.proto.RaftProtos.*;
 import org.apache.ratis.util.Preconditions;
 
 import java.util.Objects;
 
-import static org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto.TypeCase.*;
+import static org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase.*;
 
 /**
  * Request from client to server

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupId.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupId.java
index f850bec..17b48d9 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupId.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupId.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 
 import java.util.UUID;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java
index 9d81b7a..0d923ec 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java
index 06ad836..7ca4504 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.Preconditions;
 
 import java.nio.charset.StandardCharsets;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/protocol/ServerInformationReply.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/ServerInformationReply.java b/ratis-common/src/main/java/org/apache/ratis/protocol/ServerInformationReply.java
index feb326e..18bf1c0 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/ServerInformationReply.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/ServerInformationReply.java
@@ -17,8 +17,8 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.shaded.proto.RaftProtos.RoleInfoProto;
-import org.apache.ratis.shaded.proto.RaftProtos.CommitInfoProto;
+import org.apache.ratis.proto.RaftProtos.RoleInfoProto;
+import org.apache.ratis.proto.RaftProtos.CommitInfoProto;
 
 import java.util.Collection;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java b/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java
index ba4551a..8fc3581 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.util;
 
-import org.apache.ratis.shaded.com.google.common.base.Supplier;
+import org.apache.ratis.thirdparty.com.google.common.base.Supplier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java
index df0fab7..d1ef2e0 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java
@@ -18,10 +18,10 @@
 package org.apache.ratis.util;
 
 import org.apache.ratis.protocol.*;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ServiceException;
+import org.apache.ratis.proto.RaftProtos.*;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase;
 
 import java.io.IOException;
 import java.io.ObjectInputStream;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java
index 70039b2..6a9442e 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java
@@ -17,9 +17,9 @@
  */
 package org.apache.ratis.util;
 
-import org.apache.ratis.shaded.com.google.common.collect.Interner;
-import org.apache.ratis.shaded.com.google.common.collect.Interners;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.common.collect.Interner;
+import org.apache.ratis.thirdparty.com.google.common.collect.Interners;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml
index 9a7183f..16cb5b7 100644
--- a/ratis-examples/pom.xml
+++ b/ratis-examples/pom.xml
@@ -25,7 +25,11 @@
 
   <dependencies>
     <dependency>
-      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-proto</artifactId>
       <groupId>org.apache.ratis</groupId>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
index 9b5c518..f0cb359 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
@@ -24,8 +24,8 @@ import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.RaftStorage;
-import org.apache.ratis.shaded.proto.RaftProtos;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.StateMachineStorage;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.statemachine.impl.BaseStateMachine;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/AssignmentMessage.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/AssignmentMessage.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/AssignmentMessage.java
index 097e792..f8ea663 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/AssignmentMessage.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/AssignmentMessage.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import org.apache.ratis.examples.arithmetic.expression.Expression;
 import org.apache.ratis.examples.arithmetic.expression.Variable;
 import org.apache.ratis.protocol.Message;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 
 public class AssignmentMessage implements Message, Evaluable {
   public static final Charset UTF8 = Charset.forName("UTF-8");

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Assign.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Assign.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Assign.java
index a874b7a..be45c03 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Assign.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Assign.java
@@ -23,7 +23,7 @@ import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.examples.arithmetic.AssignmentMessage;
 import org.apache.ratis.examples.arithmetic.expression.*;
 import org.apache.ratis.protocol.RaftClientReply;
-import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -101,4 +101,4 @@ public class Assign extends Client {
     }
   }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Client.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Client.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Client.java
index b83b385..f7647bd 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Client.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Client.java
@@ -24,7 +24,7 @@ import org.apache.ratis.grpc.GrpcFactory;
 import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.protocol.RaftGroup;
 import org.apache.ratis.protocol.RaftGroupId;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Server.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Server.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Server.java
index 547fc0a..564dfcf 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Server.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Server.java
@@ -28,7 +28,7 @@ import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.NetUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java
index 636234f..cdbd361 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java
@@ -18,7 +18,7 @@
 package org.apache.ratis.examples.filestore;
 
 import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java
index aba2a19..8dda656 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java
@@ -20,8 +20,8 @@ package org.apache.ratis.examples.filestore;
 import org.apache.ratis.examples.filestore.FileInfo.ReadOnly;
 import org.apache.ratis.examples.filestore.FileInfo.UnderConstruction;
 import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.proto.ExamplesProtos.*;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.proto.ExamplesProtos.*;
 import org.apache.ratis.util.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java
index a8f141c..439dd6a 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java
@@ -23,8 +23,8 @@ import org.apache.ratis.protocol.Message;
 import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.RaftGroup;
 import org.apache.ratis.protocol.StateMachineException;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.proto.ExamplesProtos.*;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.proto.ExamplesProtos.*;
 import org.apache.ratis.util.CheckedFunction;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreCommon.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreCommon.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreCommon.java
index 8a92adf..8e223d2 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreCommon.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreCommon.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.examples.filestore;
 
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.*;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java
index e6f56f9..bf58c6e 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java
@@ -24,11 +24,11 @@ import org.apache.ratis.protocol.RaftClientRequest;
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.storage.RaftStorage;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.ratis.shaded.proto.ExamplesProtos.*;
-import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.ratis.proto.ExamplesProtos.*;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.SMLogEntryProto;
 import org.apache.ratis.statemachine.StateMachineStorage;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.statemachine.impl.BaseStateMachine;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java
index 7e9c57e..f746da2 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java
@@ -22,8 +22,8 @@ import org.apache.ratis.MiniRaftCluster;
 import org.apache.ratis.RaftTestUtil;
 import org.apache.ratis.conf.ConfUtils;
 import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.io.netty.util.internal.ThreadLocalRandom;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.io.netty.util.internal.ThreadLocalRandom;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.*;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml
index 820302f..4a3ab81 100644
--- a/ratis-grpc/pom.xml
+++ b/ratis-grpc/pom.xml
@@ -25,9 +25,12 @@
 
   <dependencies>
     <dependency>
-      <artifactId>ratis-proto-shaded</artifactId>
       <groupId>org.apache.ratis</groupId>
-      <scope>provided</scope>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-proto</artifactId>
+      <groupId>org.apache.ratis</groupId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
index 84f01c8..e8ca1ef 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
@@ -19,10 +19,10 @@ package org.apache.ratis.grpc;
 
 import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.ServerNotReadyException;
-import org.apache.ratis.shaded.io.grpc.Metadata;
-import org.apache.ratis.shaded.io.grpc.Status;
-import org.apache.ratis.shaded.io.grpc.StatusRuntimeException;
-import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
+import org.apache.ratis.thirdparty.io.grpc.Metadata;
+import org.apache.ratis.thirdparty.io.grpc.Status;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
 import org.apache.ratis.util.*;
 import org.slf4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
index a2e53bf..936ea5f 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
@@ -24,17 +24,17 @@ import org.apache.ratis.grpc.GrpcConfigKeys;
 import org.apache.ratis.grpc.GrpcUtil;
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.util.TimeoutScheduler;
-import org.apache.ratis.shaded.io.grpc.ManagedChannel;
-import org.apache.ratis.shaded.io.grpc.StatusRuntimeException;
-import org.apache.ratis.shaded.io.grpc.netty.NegotiationType;
-import org.apache.ratis.shaded.io.grpc.netty.NettyChannelBuilder;
-import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
-import org.apache.ratis.shaded.proto.grpc.AdminProtocolServiceGrpc;
-import org.apache.ratis.shaded.proto.grpc.AdminProtocolServiceGrpc.AdminProtocolServiceBlockingStub;
-import org.apache.ratis.shaded.proto.grpc.RaftClientProtocolServiceGrpc;
-import org.apache.ratis.shaded.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceBlockingStub;
-import org.apache.ratis.shaded.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceStub;
+import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType;
+import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
+import org.apache.ratis.proto.RaftProtos.*;
+import org.apache.ratis.proto.grpc.AdminProtocolServiceGrpc;
+import org.apache.ratis.proto.grpc.AdminProtocolServiceGrpc.AdminProtocolServiceBlockingStub;
+import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc;
+import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceBlockingStub;
+import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceStub;
 import org.apache.ratis.util.CheckedSupplier;
 import org.apache.ratis.util.CollectionUtils;
 import org.apache.ratis.util.JavaUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java
index 156e6c3..efe19f4 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java
@@ -19,9 +19,9 @@ package org.apache.ratis.grpc.client;
 
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.protocol.ClientId;
-import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
+import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
 import org.apache.ratis.protocol.RaftPeer;
 
 import java.io.Closeable;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolService.java
index 22f7f56..a099a0a 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolService.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolService.java
@@ -20,11 +20,11 @@ package org.apache.ratis.grpc.client;
 import org.apache.ratis.client.impl.ClientProtoUtils;
 import org.apache.ratis.grpc.GrpcUtil;
 import org.apache.ratis.protocol.*;
-import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
-import org.apache.ratis.shaded.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceImplBase;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
+import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceImplBase;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.SlidingWindow;
@@ -192,4 +192,4 @@ public class GrpcClientProtocolService extends RaftClientProtocolServiceImplBase
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
index 47264e7..c1f3075 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
@@ -29,12 +29,12 @@ import org.apache.ratis.protocol.RaftClientRequest;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.protocol.ServerInformationRequest;
 import org.apache.ratis.protocol.SetConfigurationRequest;
-import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
-import org.apache.ratis.shaded.proto.RaftProtos.GroupManagementRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.ServerInformationRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
+import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto;
+import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.proto.RaftProtos.ServerInformationRequestProto;
+import org.apache.ratis.proto.RaftProtos.SetConfigurationRequestProto;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.PeerProxyMap;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientStreamer.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientStreamer.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientStreamer.java
index 5e5b941..71e39b1 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientStreamer.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientStreamer.java
@@ -22,10 +22,10 @@ import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.grpc.GrpcConfigKeys;
 import org.apache.ratis.grpc.GrpcUtil;
 import org.apache.ratis.protocol.*;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcRequestProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.proto.RaftProtos.RaftRpcRequestProto;
 import org.apache.ratis.util.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java
index 1201bf2..4fe4c1a 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java
@@ -22,12 +22,12 @@ import org.apache.ratis.grpc.GrpcUtil;
 import org.apache.ratis.protocol.AdminAsynchronousProtocol;
 import org.apache.ratis.protocol.GroupManagementRequest;
 import org.apache.ratis.protocol.ServerInformationRequest;
-import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.GroupManagementRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.ServerInformationReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.ServerInformationRequestProto;
-import org.apache.ratis.shaded.proto.grpc.AdminProtocolServiceGrpc.AdminProtocolServiceImplBase;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
+import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto;
+import org.apache.ratis.proto.RaftProtos.ServerInformationReplyProto;
+import org.apache.ratis.proto.RaftProtos.ServerInformationRequestProto;
+import org.apache.ratis.proto.grpc.AdminProtocolServiceGrpc.AdminProtocolServiceImplBase;
 
 public class GrpcAdminProtocolService extends AdminProtocolServiceImplBase {
   private final AdminAsynchronousProtocol protocol;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
index 3da58bf..66c9948 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
@@ -25,11 +25,11 @@ import org.apache.ratis.server.impl.LeaderState;
 import org.apache.ratis.server.impl.LogAppender;
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.impl.ServerProtoUtils;
-import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.util.*;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java
index 3b2f8ba..0848e65 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java
@@ -17,13 +17,13 @@
  */
 package org.apache.ratis.grpc.server;
 
-import org.apache.ratis.shaded.io.grpc.ManagedChannel;
-import org.apache.ratis.shaded.io.grpc.netty.NettyChannelBuilder;
-import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
-import org.apache.ratis.shaded.proto.grpc.RaftServerProtocolServiceGrpc;
-import org.apache.ratis.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceBlockingStub;
-import org.apache.ratis.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceStub;
+import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
+import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
+import org.apache.ratis.proto.RaftProtos.*;
+import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc;
+import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceBlockingStub;
+import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceStub;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.util.TimeDuration;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java
index 83335b8..12a717a 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java
@@ -20,9 +20,9 @@ package org.apache.ratis.grpc.server;
 import org.apache.ratis.grpc.GrpcUtil;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.RaftServer;
-import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
-import org.apache.ratis.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
+import org.apache.ratis.proto.RaftProtos.*;
+import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase;
 import org.apache.ratis.util.ProtoUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java
index eb8310c..9c94cca 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java
@@ -25,9 +25,9 @@ import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.RaftServerRpc;
 import org.apache.ratis.server.impl.RaftServerRpcWithProxy;
-import org.apache.ratis.shaded.io.grpc.Server;
-import org.apache.ratis.shaded.io.grpc.netty.NettyServerBuilder;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.thirdparty.io.grpc.Server;
+import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder;
+import org.apache.ratis.proto.RaftProtos.*;
 import org.apache.ratis.util.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
index f3897ac..631bc1a 100644
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
@@ -27,7 +27,7 @@ import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.shaded.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.util.LogUtils;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.util.StringUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
index 30a3f0d..cd04b43 100644
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
@@ -34,7 +34,7 @@ import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.protocol.RaftClientRequest;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.shaded.proto.RaftProtos;
+import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.util.LogUtils;
 import org.junit.Assert;
 import org.junit.Test;
@@ -112,4 +112,4 @@ public class TestRetryCacheWithGrpc extends RetryCacheTests {
     }
     return futures;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-hadoop-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-hadoop-shaded/pom.xml b/ratis-hadoop-shaded/pom.xml
index 57f4c6c..28fdd43 100644
--- a/ratis-hadoop-shaded/pom.xml
+++ b/ratis-hadoop-shaded/pom.xml
@@ -39,6 +39,72 @@
           <skipAssembly>true</skipAssembly>
         </configuration>
       </plugin>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <!-- Replace the "default" jar-plugin execution -->
+            <!-- This is a super-dirty hack to work around Yetus
+                 PreCommit not using the package lifecycle phase -->
+            <id>default-jar</id>
+            <phase>process-classes</phase>
+            <goals>
+              <goal>jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>process-classes</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <shadeSourcesContent>true</shadeSourcesContent>
+              <createSourcesJar>true</createSourcesJar>
+              <!-- Replace the original artifact which is no good on its own -->
+              <shadedArtifactAttached>false</shadedArtifactAttached>
+              <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml</dependencyReducedPomLocation>
+              <!-- Using shade-plugin to relocate hadoop's protobuf dependency and find it in ratis-thirdparty.
+                   So dirty. -->
+              <relocations>
+                <relocation>
+                  <pattern>com.google.protobuf</pattern>
+                  <shadedPattern>org.apache.ratis.thirdparty.com.google.protobuf</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hadoop.ipc.protobuf</pattern>
+                  <shadedPattern>org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf</shadedPattern>
+                </relocation>
+              </relocations>
+
+              <filters>
+                <filter>
+                  <artifact>org.apache.hadoop:hadoop-common</artifact>
+                  <includes>
+                    <include>org/apache/hadoop/ipc/protobuf/**</include>
+                  </includes>
+                </filter>
+              </filters>
+
+              <artifactSet>
+                <includes>
+                  <!--  Must list explicitly, otherwise we pull in all of hadoop-common's
+                        transitive dependencies. -->
+                  <include>org.apache.hadoop:hadoop-common</include>
+                  <include>com.google.j2objc:j2objc-annotations</include>
+                  <include>com.google.errorprone:error_prone_annotations</include>
+                  <include>org.codehaus.mojo:animal-sniffer-annotations</include>
+                </includes>
+              </artifactSet>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     <pluginManagement>
       <plugins>
@@ -73,6 +139,10 @@
 
   <dependencies>
     <dependency>
+      <groupId>org.apache.ratis</groupId>
+      <artifactId>ratis-thirdparty</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <version>${shaded.hadoop.version}</version>
@@ -107,74 +177,6 @@
           <name>!skipShade</name>
         </property>
       </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <artifactId>maven-jar-plugin</artifactId>
-            <executions>
-              <execution>
-                <!-- Replace the "default" jar-plugin execution -->
-                <!-- This is a super-dirty hack to work around Yetus
-                     PreCommit not using the package lifecycle phase -->
-                <id>default-jar</id>
-                <phase>process-classes</phase>
-                <goals>
-                  <goal>jar</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-shade-plugin</artifactId>
-            <executions>
-              <execution>
-                <phase>process-classes</phase>
-                <goals>
-                  <goal>shade</goal>
-                </goals>
-                <configuration>
-                  <shadeSourcesContent>true</shadeSourcesContent>
-                  <createSourcesJar>true</createSourcesJar>
-                  <!-- Replace the original artifact which is no good on its own -->
-                  <shadedArtifactAttached>false</shadedArtifactAttached>
-                  <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml</dependencyReducedPomLocation>
-                  <relocations>
-                    <relocation>
-                      <pattern>com.google.protobuf</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.com.google.protobuf</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>org.apache.hadoop.ipc.protobuf</pattern>
-                      <shadedPattern>org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf</shadedPattern>
-                    </relocation>
-                  </relocations>
-
-                  <filters>
-                    <filter>
-                      <artifact>org.apache.hadoop:hadoop-common</artifact>
-                      <includes>
-                        <include>org/apache/hadoop/ipc/protobuf/**</include>
-                      </includes>
-                    </filter>
-                  </filters>
-
-                  <artifactSet>
-                    <includes>
-                      <!--  Must list explicitly, otherwise we pull in all of hadoop-common's
-                            transitive dependencies. -->
-                      <include>org.apache.hadoop:hadoop-common</include>
-                      <include>com.google.j2objc:j2objc-annotations</include>
-                      <include>com.google.errorprone:error_prone_annotations</include>
-                      <include>org.codehaus.mojo:animal-sniffer-annotations</include>
-                    </includes>
-                  </artifactSet>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
     </profile>
   </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-hadoop/pom.xml b/ratis-hadoop/pom.xml
index a61e786..c98fe26 100644
--- a/ratis-hadoop/pom.xml
+++ b/ratis-hadoop/pom.xml
@@ -25,14 +25,12 @@
 
   <dependencies>
     <dependency>
-      <artifactId>ratis-proto-shaded</artifactId>
+      <artifactId>ratis-proto</artifactId>
       <groupId>org.apache.ratis</groupId>
-      <scope>provided</scope>
     </dependency>
     <dependency>
       <artifactId>ratis-hadoop-shaded</artifactId>
       <groupId>org.apache.ratis</groupId>
-      <scope>provided</scope>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java b/ratis-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
index cf9c917..b8fd2f0 100644
--- a/ratis-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
+++ b/ratis-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
@@ -33,8 +33,8 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.Time;
-import org.apache.ratis.shaded.com.google.protobuf.*;
-import org.apache.ratis.shaded.com.google.protobuf.Descriptors.MethodDescriptor;
+import org.apache.ratis.thirdparty.com.google.protobuf.*;
+import org.apache.ratis.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
 import org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf.ProtobufRpcEngineProtos.RequestHeaderProto;
 import org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
 import org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java
index 43fcca1..053df49 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java
@@ -27,7 +27,7 @@ import org.apache.ratis.protocol.GroupManagementRequest;
 import org.apache.ratis.protocol.ServerInformationRequest;
 import org.apache.ratis.protocol.ServerInformationReply;
 import org.apache.ratis.protocol.SetConfigurationRequest;
-import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
+import org.apache.ratis.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.ratis.util.CheckedFunction;
 import org.apache.ratis.util.ProtoUtils;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolPB.java
index e9af3b0..28e6b54 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolPB.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolPB.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.ratis.hadooprpc.HadoopConstants;
-import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.CombinedClientProtocolService;
+import org.apache.ratis.proto.hadoop.HadoopProtos.CombinedClientProtocolService;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolServerSideTranslatorPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolServerSideTranslatorPB.java
index c47e5b0..77fc8ef 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolServerSideTranslatorPB.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolServerSideTranslatorPB.java
@@ -23,14 +23,14 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.ratis.client.impl.ClientProtoUtils;
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.server.RaftServer;
-import org.apache.ratis.shaded.com.google.protobuf.RpcController;
-import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.GroupManagementRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.ServerInformationRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.ServerInformationReplyProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.RpcController;
+import org.apache.ratis.thirdparty.com.google.protobuf.ServiceException;
+import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto;
+import org.apache.ratis.proto.RaftProtos.ServerInformationRequestProto;
+import org.apache.ratis.proto.RaftProtos.ServerInformationReplyProto;
 
 @InterfaceAudience.Private
 public class CombinedClientProtocolServerSideTranslatorPB

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
index 5d07752..891bf73 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
@@ -30,12 +30,12 @@ import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerRpc;
 import org.apache.ratis.server.impl.RaftServerRpcWithProxy;
 import org.apache.ratis.server.protocol.RaftServerProtocol;
-import org.apache.ratis.shaded.com.google.protobuf.BlockingService;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
-import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.CombinedClientProtocolService;
-import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.RaftServerProtocolService;
+import org.apache.ratis.thirdparty.com.google.protobuf.BlockingService;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.ServiceException;
+import org.apache.ratis.proto.RaftProtos.*;
+import org.apache.ratis.proto.hadoop.HadoopProtos.CombinedClientProtocolService;
+import org.apache.ratis.proto.hadoop.HadoopProtos.RaftServerProtocolService;
 import org.apache.ratis.util.CheckedFunction;
 import org.apache.ratis.util.CodeInjectionForTesting;
 import org.apache.ratis.util.PeerProxyMap;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolPB.java
index 8b92cc4..69ad219 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolPB.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolPB.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.ratis.hadooprpc.HadoopConstants;
-import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.RaftServerProtocolService;
+import org.apache.ratis.proto.hadoop.HadoopProtos.RaftServerProtocolService;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
index a496793..85ada23 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
@@ -21,14 +21,14 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.ratis.server.protocol.RaftServerProtocol;
-import org.apache.ratis.shaded.com.google.protobuf.RpcController;
-import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.RpcController;
+import org.apache.ratis.thirdparty.com.google.protobuf.ServiceException;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto;
 
 @InterfaceAudience.Private
 public class RaftServerProtocolServerSideTranslatorPB

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-logservice/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-logservice/pom.xml b/ratis-logservice/pom.xml
index bfe002e..16ff962 100644
--- a/ratis-logservice/pom.xml
+++ b/ratis-logservice/pom.xml
@@ -26,7 +26,7 @@
   <dependencies>
     <!-- Ratis dependencies -->
     <dependency>
-      <artifactId>ratis-proto-shaded</artifactId>
+      <artifactId>ratis-proto</artifactId>
       <groupId>org.apache.ratis</groupId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/3b9d50de/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogMessage.java
----------------------------------------------------------------------
diff --git a/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogMessage.java b/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogMessage.java
index bf2024f..cfc53a5 100644
--- a/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogMessage.java
+++ b/ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogMessage.java
@@ -20,10 +20,10 @@ package org.apache.ratis.logservice.api;
 import java.nio.charset.Charset;
 
 import org.apache.ratis.protocol.Message;
-import org.apache.ratis.shaded.proto.logservice.LogServiceProtos;
+import org.apache.ratis.proto.logservice.LogServiceProtos;
 
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
 
 public class LogMessage implements Message {
   public static final Charset UTF8 = Charset.forName("UTF-8");