You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by hi...@apache.org on 2017/06/26 16:26:55 UTC

[08/13] geode git commit: GEODE-2995: Handle stream of ProtoBuf encoded messages

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.protocol.operations.OperationHandler
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.protocol.operations.OperationHandler b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.protocol.operations.OperationHandler
new file mode 100644
index 0000000..fa7c3c6
--- /dev/null
+++ b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.protocol.operations.OperationHandler
@@ -0,0 +1 @@
+org.apache.geode.protocol.operations.protobuf.GetRequestOperationHandler
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.serialization.TypeCodec
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.serialization.TypeCodec b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.serialization.TypeCodec
new file mode 100644
index 0000000..1b7b333
--- /dev/null
+++ b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.serialization.TypeCodec
@@ -0,0 +1,10 @@
+org.apache.geode.serialization.codec.BinaryCodec
+org.apache.geode.serialization.codec.BooleanCodec
+org.apache.geode.serialization.codec.ByteCodec
+org.apache.geode.serialization.codec.DoubleCodec
+org.apache.geode.serialization.codec.FloatCodec
+org.apache.geode.serialization.codec.IntCodec
+org.apache.geode.serialization.codec.JSONCodec
+org.apache.geode.serialization.codec.LongCodec
+org.apache.geode.serialization.codec.ShortCodec
+org.apache.geode.serialization.codec.StringCodec

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/client/protocol/IntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/client/protocol/IntegrationTest.java b/geode-protobuf/src/test/java/org/apache/geode/client/protocol/IntegrationTest.java
new file mode 100644
index 0000000..e86e908
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/client/protocol/IntegrationTest.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.client.protocol;
+
+import com.google.protobuf.ByteString;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.protocol.exception.InvalidProtocolMessageException;
+import org.apache.geode.protocol.handler.ProtobufStreamProcessor;
+import org.apache.geode.protocol.operations.registry.exception.OperationHandlerAlreadyRegisteredException;
+import org.apache.geode.protocol.operations.registry.exception.OperationHandlerNotRegisteredException;
+import org.apache.geode.protocol.protobuf.BasicTypes;
+import org.apache.geode.protocol.protobuf.ClientProtocol;
+import org.apache.geode.protocol.protobuf.RegionAPI;
+import org.apache.geode.serialization.codec.StringCodec;
+import org.apache.geode.serialization.protobuf.translation.EncodingTypeTranslator;
+import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
+import org.apache.geode.serialization.registry.SerializationCodecRegistry;
+import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
+import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
+import org.apache.geode.test.dunit.Assert;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+@Category(UnitTest.class)
+public class IntegrationTest {
+
+  public static final String TEST_KEY = "my key";
+  public static final String TEST_VALUE = "my value";
+  public static final String TEST_REGION = "test region";
+  private StringCodec stringCodec;
+  private Cache cacheStub;
+
+  @Before
+  public void setup() throws CodecAlreadyRegisteredForTypeException,
+      UnsupportedEncodingTypeException, CodecNotRegisteredForTypeException {
+    SerializationCodecRegistry serializationCodecRegistry = new SerializationCodecRegistry();
+    stringCodec = (StringCodec) serializationCodecRegistry.getCodecForType(
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.STRING));
+    cacheStub = getCacheStub();
+  }
+
+  @Test
+  public void testFullRequestToCache()
+      throws OperationHandlerAlreadyRegisteredException, CodecAlreadyRegisteredForTypeException,
+      UnsupportedEncodingTypeException, CodecNotRegisteredForTypeException,
+      OperationHandlerNotRegisteredException, IOException, InvalidProtocolMessageException {
+
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+
+    ProtobufStreamProcessor streamProcessor = new ProtobufStreamProcessor();
+    streamProcessor.processOneMessage(getInputStream(getRequest(stringCodec)), outputStream,
+        cacheStub);
+
+    RegionAPI.GetResponse getResponse = getGetResponse(outputStream);
+
+    Assert.assertNotNull(getResponse);
+    Assert.assertEquals(BasicTypes.EncodingType.STRING, getResponse.getResult().getEncodingType());
+    String actualValue = stringCodec.decode(getResponse.getResult().getValue().toByteArray());
+    Assert.assertEquals(TEST_VALUE, actualValue);
+  }
+
+  private Cache getCacheStub() {
+    Region regionStub = mock(Region.class);
+    when(regionStub.get(TEST_KEY)).thenReturn(TEST_VALUE);
+
+    Cache cacheStub = mock(Cache.class);
+    when(cacheStub.getRegion(TEST_REGION)).thenReturn(regionStub);
+    return cacheStub;
+  }
+
+  private RegionAPI.GetResponse getGetResponse(ByteArrayOutputStream outputStream)
+      throws IOException {
+    ByteArrayInputStream helperInputStream = new ByteArrayInputStream(outputStream.toByteArray());
+    ClientProtocol.Message responseMessage =
+        ClientProtocol.Message.parseDelimitedFrom(helperInputStream);
+    ClientProtocol.Response response = responseMessage.getResponse();
+    return response.getGetResponse();
+  }
+
+  private ByteArrayInputStream getInputStream(ClientProtocol.Message request) throws IOException {
+    ByteArrayOutputStream helperOutputStream = new ByteArrayOutputStream();
+    request.writeDelimitedTo(helperOutputStream);
+    return new ByteArrayInputStream(helperOutputStream.toByteArray());
+  }
+
+  private ClientProtocol.Message getRequest(StringCodec stringCodec) {
+    RegionAPI.GetRequest.Builder getRequestBuilder = RegionAPI.GetRequest.newBuilder();
+    getRequestBuilder.setRegionName(TEST_REGION)
+        .setKey(BasicTypes.EncodedValue.newBuilder().setEncodingType(BasicTypes.EncodingType.STRING)
+            .setValue(ByteString.copyFrom(stringCodec.encode(TEST_KEY))));
+    ClientProtocol.Request request =
+        ClientProtocol.Request.newBuilder().setGetRequest(getRequestBuilder).build();
+    ClientProtocol.Message requestMessage = ClientProtocol.Message.newBuilder()
+        .setMessageHeader(ClientProtocol.MessageHeader.newBuilder()).setRequest(request).build();
+
+    return requestMessage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/client/protocol/MessageUtil.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/client/protocol/MessageUtil.java b/geode-protobuf/src/test/java/org/apache/geode/client/protocol/MessageUtil.java
new file mode 100644
index 0000000..c830b01
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/client/protocol/MessageUtil.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.client.protocol;
+
+import org.apache.geode.protocol.protobuf.ClientProtocol;
+import org.apache.geode.protocol.protobuf.RegionAPI;
+
+public class MessageUtil {
+  public static ClientProtocol.Message createGetRequestMessage() {
+    ClientProtocol.Message.Builder messageBuilder = ClientProtocol.Message.newBuilder();
+    messageBuilder.setMessageHeader(getMessageHeaderBuilder());
+    ClientProtocol.Request.Builder requestBuilder = getRequestBuilder();
+    requestBuilder.setGetRequest(getGetRequestBuilder());
+    messageBuilder.setRequest(requestBuilder);
+    return messageBuilder.build();
+  }
+
+  private static ClientProtocol.Request.Builder getRequestBuilder() {
+    return ClientProtocol.Request.newBuilder();
+  }
+
+  private static RegionAPI.GetRequest.Builder getGetRequestBuilder() {
+    return RegionAPI.GetRequest.newBuilder();
+  }
+
+  private static ClientProtocol.MessageHeader.Builder getMessageHeaderBuilder() {
+    return ClientProtocol.MessageHeader.newBuilder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/client/protocol/OpsHandler.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/client/protocol/OpsHandler.java b/geode-protobuf/src/test/java/org/apache/geode/client/protocol/OpsHandler.java
new file mode 100644
index 0000000..793dace
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/client/protocol/OpsHandler.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.client.protocol;
+
+import org.apache.geode.serialization.registry.SerializationCodecRegistry;
+
+public interface OpsHandler<Req, Resp> {
+  Resp process(SerializationCodecRegistry serializationCodecRegistry, Req request);
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/client/protocol/OpsProcessorTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/client/protocol/OpsProcessorTest.java b/geode-protobuf/src/test/java/org/apache/geode/client/protocol/OpsProcessorTest.java
new file mode 100644
index 0000000..1ec7a02
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/client/protocol/OpsProcessorTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.client.protocol;
+
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.RegionService;
+import org.apache.geode.protocol.OpsProcessor;
+import org.apache.geode.protocol.operations.OperationHandler;
+import org.apache.geode.protocol.operations.ProtobufRequestOperationParser;
+import org.apache.geode.protocol.operations.registry.OperationsHandlerRegistry;
+import org.apache.geode.protocol.operations.registry.exception.OperationHandlerNotRegisteredException;
+import org.apache.geode.protocol.protobuf.ClientProtocol;
+import org.apache.geode.protocol.protobuf.RegionAPI;
+import org.apache.geode.serialization.SerializationService;
+import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
+import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class OpsProcessorTest {
+  @Test
+  public void testOpsProcessor() throws CodecNotRegisteredForTypeException,
+      OperationHandlerNotRegisteredException, UnsupportedEncodingTypeException {
+    OperationsHandlerRegistry opsHandlerRegistryStub = mock(OperationsHandlerRegistry.class);
+    OperationHandler operationHandlerStub = mock(OperationHandler.class);
+    SerializationService serializationServiceStub = mock(SerializationService.class);
+    Cache dummyCache = mock(Cache.class);
+    int operationID = ClientProtocol.Request.RequestAPICase.GETREQUEST.getNumber();
+
+    ClientProtocol.Request messageRequest = ClientProtocol.Request.newBuilder()
+        .setGetRequest(RegionAPI.GetRequest.newBuilder()).build();
+
+    RegionAPI.GetResponse expectedResponse = RegionAPI.GetResponse.newBuilder().build();
+
+    when(opsHandlerRegistryStub.getOperationHandlerForOperationId(operationID))
+        .thenReturn(operationHandlerStub);
+    when(operationHandlerStub.process(serializationServiceStub,
+        ProtobufRequestOperationParser.getRequestForOperationTypeID(messageRequest), dummyCache))
+            .thenReturn(expectedResponse);
+
+    OpsProcessor processor = new OpsProcessor(opsHandlerRegistryStub, serializationServiceStub);
+    ClientProtocol.Response response = processor.process(messageRequest, dummyCache);
+    Assert.assertEquals(expectedResponse, response.getGetResponse());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/protocol/handler/ProtobufProtocolHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/handler/ProtobufProtocolHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/handler/ProtobufProtocolHandlerJUnitTest.java
new file mode 100644
index 0000000..21c3107
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/handler/ProtobufProtocolHandlerJUnitTest.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.protocol.handler;
+
+import org.apache.geode.client.protocol.MessageUtil;
+import org.apache.geode.protocol.exception.InvalidProtocolMessageException;
+import org.apache.geode.protocol.handler.ProtocolHandler;
+import org.apache.geode.protocol.handler.protobuf.ProtobufProtocolHandler;
+import org.apache.geode.protocol.protobuf.ClientProtocol;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ServiceLoader;
+
+@Category(UnitTest.class)
+public class ProtobufProtocolHandlerJUnitTest {
+  private ProtocolHandler<ClientProtocol.Message> protocolHandler;
+
+  @Before
+  public void startup() {
+    ServiceLoader<ProtocolHandler> serviceLoader = ServiceLoader.load(ProtocolHandler.class);
+    for (ProtocolHandler protocolHandler : serviceLoader) {
+      if (protocolHandler instanceof ProtobufProtocolHandler) {
+        this.protocolHandler = protocolHandler;
+      }
+    }
+  }
+
+  @Test
+  public void testDeserializeByteArrayToMessage()
+      throws IOException, InvalidProtocolMessageException {
+    ClientProtocol.Message expectedRequestMessage = MessageUtil.createGetRequestMessage();
+
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+
+    expectedRequestMessage.writeDelimitedTo(byteArrayOutputStream);
+    InputStream inputStream = new ByteArrayInputStream(byteArrayOutputStream.toByteArray());
+
+    ClientProtocol.Message actualMessage = protocolHandler.deserialize(inputStream);
+    Assert.assertEquals(expectedRequestMessage, actualMessage);
+  }
+
+  @Test
+  public void testDeserializeInvalidByteThrowsException() throws IOException {
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    byteArrayOutputStream.write("Some incorrect byte array".getBytes());
+    InputStream inputStream = new ByteArrayInputStream(byteArrayOutputStream.toByteArray());
+
+    boolean caughtException = false;
+    try {
+      protocolHandler.deserialize(inputStream);
+    } catch (InvalidProtocolMessageException e) {
+      caughtException = true;
+    }
+    Assert.assertTrue(caughtException);
+  }
+
+  @Test
+  public void testSerializeMessageToByteArray() throws IOException {
+    ClientProtocol.Message message = MessageUtil.createGetRequestMessage();
+    ByteArrayOutputStream expectedByteArrayOutputStream = new ByteArrayOutputStream();
+    message.writeDelimitedTo(expectedByteArrayOutputStream);
+    byte[] expectedByteArray = expectedByteArrayOutputStream.toByteArray();
+
+    ByteArrayOutputStream actualByteArrayOutputStream = new ByteArrayOutputStream();
+    protocolHandler.serialize(message, actualByteArrayOutputStream);
+    Assert.assertArrayEquals(expectedByteArray, actualByteArrayOutputStream.toByteArray());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/protocol/operations/protobuf/GetRequestOperationHandlerTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/operations/protobuf/GetRequestOperationHandlerTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/operations/protobuf/GetRequestOperationHandlerTest.java
new file mode 100644
index 0000000..359db7d
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/operations/protobuf/GetRequestOperationHandlerTest.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.protocol.operations.protobuf;
+
+import com.google.protobuf.ByteString;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionService;
+import org.apache.geode.protocol.protobuf.BasicTypes;
+import org.apache.geode.protocol.protobuf.RegionAPI;
+import org.apache.geode.serialization.SerializationService;
+import org.apache.geode.serialization.codec.StringCodec;
+import org.apache.geode.serialization.protobuf.translation.EncodingTypeTranslator;
+import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
+import org.apache.geode.serialization.registry.SerializationCodecRegistry;
+import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
+import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
+import org.apache.geode.test.dunit.Assert;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.charset.Charset;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@Category(UnitTest.class)
+public class GetRequestOperationHandlerTest {
+  public static final String TEST_KEY = "my key";
+  public static final String TEST_VALUE = "my value";
+  public static final String TEST_REGION = "test region";
+  public Cache cacheStub;
+  public SerializationService serializationServiceStub;
+
+  @Before
+  public void setUp() throws Exception {
+    serializationServiceStub = mock(SerializationService.class);
+    when(serializationServiceStub.decode(BasicTypes.EncodingType.STRING,
+        TEST_KEY.getBytes(Charset.forName("UTF-8")))).thenReturn(TEST_KEY);
+    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, TEST_VALUE))
+        .thenReturn(TEST_VALUE.getBytes(Charset.forName("UTF-8")));
+
+    Region regionStub = mock(Region.class);
+    when(regionStub.get(TEST_KEY)).thenReturn(TEST_VALUE);
+
+    cacheStub = mock(Cache.class);
+    when(cacheStub.getRegion(TEST_REGION)).thenReturn(regionStub);
+  }
+
+  @Test
+  public void processReturnsTheEncodedValueFromTheRegion() throws UnsupportedEncodingTypeException,
+      CodecNotRegisteredForTypeException, CodecAlreadyRegisteredForTypeException {
+    GetRequestOperationHandler operationHandler = new GetRequestOperationHandler();
+
+    RegionAPI.GetResponse response =
+        operationHandler.process(serializationServiceStub, makeGetRequest(), cacheStub);
+
+    Assert.assertEquals(BasicTypes.EncodingType.STRING, response.getResult().getEncodingType());
+    String actualValue = getStringCodec().decode(response.getResult().getValue().toByteArray());
+    Assert.assertEquals(TEST_VALUE, actualValue);
+  }
+
+  private RegionAPI.GetRequest makeGetRequest() throws CodecNotRegisteredForTypeException,
+      UnsupportedEncodingTypeException, CodecAlreadyRegisteredForTypeException {
+    StringCodec stringCodec = getStringCodec();
+    RegionAPI.GetRequest.Builder getRequestBuilder = RegionAPI.GetRequest.newBuilder();
+    getRequestBuilder.setRegionName(TEST_REGION)
+        .setKey(BasicTypes.EncodedValue.newBuilder().setEncodingType(BasicTypes.EncodingType.STRING)
+            .setValue(ByteString.copyFrom(stringCodec.encode(TEST_KEY))));
+
+    return getRequestBuilder.build();
+  }
+
+  private StringCodec getStringCodec() throws CodecAlreadyRegisteredForTypeException,
+      CodecNotRegisteredForTypeException, UnsupportedEncodingTypeException {
+    SerializationCodecRegistry serializationCodecRegistry = new SerializationCodecRegistry();
+    return (StringCodec) serializationCodecRegistry.getCodecForType(
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.STRING));
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/protocol/operations/registry/OperationsHandlerRegistryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/operations/registry/OperationsHandlerRegistryJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/operations/registry/OperationsHandlerRegistryJUnitTest.java
new file mode 100644
index 0000000..612e6a7
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/operations/registry/OperationsHandlerRegistryJUnitTest.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.protocol.operations.registry;
+
+import static io.codearte.catchexception.shade.mockito.Mockito.mock;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.geode.protocol.operations.OperationHandler;
+import org.apache.geode.protocol.operations.registry.exception.OperationHandlerAlreadyRegisteredException;
+import org.apache.geode.protocol.operations.registry.exception.OperationHandlerNotRegisteredException;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class OperationsHandlerRegistryJUnitTest {
+  public static final int DUMMY_OPERATION_CODE = 999;
+  private OperationsHandlerRegistry operationsHandlerRegistry;
+
+  @Before
+  public void setup() throws OperationHandlerAlreadyRegisteredException {
+    operationsHandlerRegistry = new OperationsHandlerRegistry();
+  }
+
+  @Test
+  public void testAddOperationsHandlerForOperationType()
+      throws OperationHandlerAlreadyRegisteredException {
+    int initialHandlerCount = operationsHandlerRegistry.getRegisteredOperationHandlersCount();
+    operationsHandlerRegistry.registerOperationHandlerForOperationId(DUMMY_OPERATION_CODE,
+        mock(OperationHandler.class));
+    assertEquals(initialHandlerCount + 1,
+        operationsHandlerRegistry.getRegisteredOperationHandlersCount());
+  }
+
+  @Test
+  public void testAddingDuplicateOperationsHandlerForOperationType_ThrowsException()
+      throws OperationHandlerAlreadyRegisteredException, OperationHandlerNotRegisteredException {
+    OperationHandler expectedOperationHandler = mock(OperationHandler.class);
+    OperationHandler unexpectedOperationHandler = mock(OperationHandler.class);
+    operationsHandlerRegistry.registerOperationHandlerForOperationId(DUMMY_OPERATION_CODE,
+        expectedOperationHandler);
+    int initialHandlerCount = operationsHandlerRegistry.getRegisteredOperationHandlersCount();
+    boolean exceptionCaught = false;
+    try {
+      operationsHandlerRegistry.registerOperationHandlerForOperationId(DUMMY_OPERATION_CODE,
+          unexpectedOperationHandler);
+    } catch (OperationHandlerAlreadyRegisteredException e) {
+      exceptionCaught = true;
+    }
+    assertTrue(exceptionCaught);
+    assertEquals(initialHandlerCount,
+        operationsHandlerRegistry.getRegisteredOperationHandlersCount());
+    assertSame(expectedOperationHandler,
+        operationsHandlerRegistry.getOperationHandlerForOperationId(DUMMY_OPERATION_CODE));
+  }
+
+  @Test
+  public void testGetOperationsHandlerForOperationType()
+      throws OperationHandlerAlreadyRegisteredException, OperationHandlerNotRegisteredException {
+    OperationHandler expectedOperationHandler = mock(OperationHandler.class);
+
+    operationsHandlerRegistry.registerOperationHandlerForOperationId(DUMMY_OPERATION_CODE,
+        expectedOperationHandler);
+    OperationHandler operationHandler =
+        operationsHandlerRegistry.getOperationHandlerForOperationId(DUMMY_OPERATION_CODE);
+    assertSame(expectedOperationHandler, operationHandler);
+  }
+
+  @Test
+  public void testGetOperationsHandlerForMissingOperationType_ThrowsException() {
+    boolean exceptionCaught = false;
+    try {
+      operationsHandlerRegistry.getOperationHandlerForOperationId(DUMMY_OPERATION_CODE);
+    } catch (OperationHandlerNotRegisteredException e) {
+      exceptionCaught = true;
+    }
+    assertTrue(exceptionCaught);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/serialization/ProtobufSerializationServiceImplTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/serialization/ProtobufSerializationServiceImplTest.java b/geode-protobuf/src/test/java/org/apache/geode/serialization/ProtobufSerializationServiceImplTest.java
new file mode 100644
index 0000000..a4c0a14
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/serialization/ProtobufSerializationServiceImplTest.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.serialization;
+
+import org.apache.geode.protocol.protobuf.BasicTypes;
+import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
+import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
+import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class ProtobufSerializationServiceImplTest {
+
+  public static final String PAYLOAD = "my value";
+
+  @Test
+  public void valuesPreservedByEncodingThenDecoding() throws CodecAlreadyRegisteredForTypeException,
+      UnsupportedEncodingTypeException, CodecNotRegisteredForTypeException {
+    ProtobufSerializationService protobufSerializationService = new ProtobufSerializationService();
+
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.STRING, "testString");
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.FLOAT, (float) 34.23);
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.DOUBLE, 34.23);
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.INT, 45);
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.SHORT, (short) 45);
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.BYTE, (byte) 45);
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.LONG, (long) 45);
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.BOOLEAN, false);
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.BOOLEAN, true);
+    testEncodeDecode(protobufSerializationService, BasicTypes.EncodingType.BINARY,
+        "testString".getBytes());
+    // TODO: Test JSON conversion
+  }
+
+  private void testEncodeDecode(ProtobufSerializationService service,
+      BasicTypes.EncodingType encodingType, Object data)
+      throws UnsupportedEncodingTypeException, CodecNotRegisteredForTypeException {
+    byte[] encodedValue = service.encode(encodingType, data);
+    Object decdoedValue = service.decode(encodingType, encodedValue);
+    Assert.assertEquals(data, decdoedValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/serialization/codec/StringCodecJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/serialization/codec/StringCodecJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/serialization/codec/StringCodecJUnitTest.java
new file mode 100644
index 0000000..8cd6029
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/serialization/codec/StringCodecJUnitTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.serialization.codec;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.charset.Charset;
+
+@Category(UnitTest.class)
+public class StringCodecJUnitTest {
+  private static final Charset UTF8 = Charset.forName("UTF-8");
+  private static final Charset UTF16 = Charset.forName("UTF-16");
+  private String testString = "Test String";
+
+  private StringCodec stringCodec;
+
+  @Before
+  public void startup() {
+    stringCodec = new StringCodec();
+  }
+
+  @Test
+  public void testStringEncoding() {
+    assertArrayEquals(testString.getBytes(UTF8), stringCodec.encode(testString));
+  }
+
+  @Test
+  public void testStringIncompatibleEncoding() {
+    byte[] expectedEncodedString = stringCodec.encode(testString);
+    byte[] incorrectEncodedString = testString.getBytes(UTF16);
+    assertNotEquals(expectedEncodedString.length, incorrectEncodedString.length);
+  }
+
+  @Test
+  public void testStringDecodingWithIncorrectEncodedString() {
+    byte[] encodedString = testString.getBytes(UTF16);
+    assertNotEquals(testString, stringCodec.decode(encodedString));
+  }
+
+  @Test
+  public void testStringDecoding() {
+    byte[] encodedString = testString.getBytes(UTF8);
+    assertEquals(testString, stringCodec.decode(encodedString));
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/serialization/protobuf/translation/EncodingTypeToSerializationTypeTranslatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/serialization/protobuf/translation/EncodingTypeToSerializationTypeTranslatorJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/serialization/protobuf/translation/EncodingTypeToSerializationTypeTranslatorJUnitTest.java
new file mode 100644
index 0000000..1af710c
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/serialization/protobuf/translation/EncodingTypeToSerializationTypeTranslatorJUnitTest.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.serialization.protobuf.translation;
+
+import static org.junit.Assert.assertSame;
+
+import org.apache.geode.protocol.protobuf.BasicTypes;
+import org.apache.geode.serialization.SerializationType;
+import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class EncodingTypeToSerializationTypeTranslatorJUnitTest {
+
+  @Test
+  public void testTranslateEncodingTypes() throws UnsupportedEncodingTypeException {
+    assertSame(SerializationType.INT,
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.INT));
+    assertSame(SerializationType.LONG,
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.LONG));
+    assertSame(SerializationType.SHORT,
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.SHORT));
+    assertSame(SerializationType.BYTE,
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.BYTE));
+    assertSame(SerializationType.BOOLEAN, EncodingTypeTranslator
+        .getSerializationTypeForEncodingType(BasicTypes.EncodingType.BOOLEAN));
+    assertSame(SerializationType.BINARY,
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.BINARY));
+    assertSame(SerializationType.FLOAT,
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.FLOAT));
+    assertSame(SerializationType.DOUBLE,
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.DOUBLE));
+    assertSame(SerializationType.STRING,
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.STRING));
+    assertSame(SerializationType.JSON,
+        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.JSON));
+  }
+
+  @Test(expected = UnsupportedEncodingTypeException.class)
+  public void testTranslateInvalidEncoding_throwsException()
+      throws UnsupportedEncodingTypeException {
+    EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.INVALID);
+  }
+
+  @Test
+  public void testAllEncodingTypeTranslations() {
+    for (BasicTypes.EncodingType encodingType : BasicTypes.EncodingType.values()) {
+      if (!(encodingType.equals(BasicTypes.EncodingType.UNRECOGNIZED)
+          || encodingType.equals(BasicTypes.EncodingType.INVALID))) {
+        try {
+          EncodingTypeTranslator.getSerializationTypeForEncodingType(encodingType);
+        } catch (UnsupportedEncodingTypeException e) {
+          e.printStackTrace();
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/geode-protobuf/src/test/java/org/apache/geode/serialization/registry/CodecRegistryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/serialization/registry/CodecRegistryJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/serialization/registry/CodecRegistryJUnitTest.java
new file mode 100644
index 0000000..2c37fb5
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/serialization/registry/CodecRegistryJUnitTest.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.serialization.registry;
+
+import org.apache.geode.serialization.SerializationType;
+import org.apache.geode.serialization.TypeCodec;
+import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
+import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+@Category(UnitTest.class)
+@RunWith(PowerMockRunner.class)
+@PowerMockIgnore("*.UnitTest")
+@PrepareForTest({SerializationType.class})
+public class CodecRegistryJUnitTest {
+  private SerializationCodecRegistry codecRegistry;
+
+  @Before
+  public void startup() throws CodecAlreadyRegisteredForTypeException {
+    codecRegistry = new SerializationCodecRegistry();
+  }
+
+  @After
+  public void tearDown() {
+    codecRegistry.shutdown();
+  }
+
+  @Test
+  public void testRegisterCodec() throws CodecAlreadyRegisteredForTypeException {
+    Assert.assertEquals(10, codecRegistry.getRegisteredCodecCount());
+    SerializationType mockSerializationType = PowerMockito.mock(SerializationType.class);
+    codecRegistry.register(mockSerializationType, new DummyTypeCodec());
+    Assert.assertEquals(11, codecRegistry.getRegisteredCodecCount());
+  }
+
+  @Test
+  public void testRegisteringCodecForRegisteredType_throwsException()
+      throws CodecAlreadyRegisteredForTypeException {
+    SerializationType mockSerializationType = PowerMockito.mock(SerializationType.class);
+    codecRegistry.register(mockSerializationType, new DummyTypeCodec());
+
+    boolean caughtException = false;
+    try {
+      codecRegistry.register(mockSerializationType, new DummyTypeCodec());
+    } catch (CodecAlreadyRegisteredForTypeException e) {
+      caughtException = true;
+    }
+    Assert.assertTrue("This was supposed to have thrown a CodecAlreadyRegisteredException",
+        caughtException);
+  }
+
+  @Test
+  public void testGetRegisteredCodec()
+      throws CodecAlreadyRegisteredForTypeException, CodecNotRegisteredForTypeException {
+    TypeCodec expectedCodec = new DummyTypeCodec();
+    SerializationType mockSerializationType = PowerMockito.mock(SerializationType.class);
+    codecRegistry.register(mockSerializationType, expectedCodec);
+    TypeCodec codec = codecRegistry.getCodecForType(mockSerializationType);
+    Assert.assertSame(expectedCodec, codec);
+  }
+
+  @Test
+  public void testGetCodecForUnregisteredType_throwsException() {
+    boolean caughtException = false;
+    try {
+      SerializationType mockSerializationType = PowerMockito.mock(SerializationType.class);
+      codecRegistry.getCodecForType(mockSerializationType);
+    } catch (CodecNotRegisteredForTypeException e) {
+      caughtException = true;
+    }
+    Assert.assertTrue("This should have thrown a CodecNotRegisteredForTypeException",
+        caughtException);
+  }
+
+  class DummyTypeCodec implements TypeCodec {
+    @Override
+    public Object decode(byte[] incoming) {
+      return null;
+    }
+
+    @Override
+    public byte[] encode(Object incoming) {
+      return new byte[0];
+    }
+
+    @Override
+    public SerializationType getSerializationType() {
+      return PowerMockito.mock(SerializationType.class);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 1bea584..a1582b9 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -113,11 +113,7 @@ rat {
     'geode-core/src/test/resources/org/apache/geode/management/internal/configuration/domain/CacheElementJUnitTest.xml',
     'geode-core/src/test/resources/org/apache/geode/management/internal/configuration/utils/*.xml',
 
-    '**/META-INF/services/org.xml.sax.ext.EntityResolver2',
-    '**/META-INF/services/org.apache.geode.internal.cache.CacheService',
-    '**/META-INF/services/org.apache.geode.internal.cache.xmlcache.XmlParser',
-    '**/META-INF/services/org.apache.geode.distributed.ServerLauncherCacheProvider',
-    '**/META-INF/services/org.springframework.shell.core.CommandMarker',
+    '**/META-INF/**',
 
     // --- Other Licenses ---
 

http://git-wip-us.apache.org/repos/asf/geode/blob/cdcc4d95/settings.gradle
----------------------------------------------------------------------
diff --git a/settings.gradle b/settings.gradle
index c0fdb6e..39697b7 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -37,8 +37,10 @@ include 'extensions/geode-modules-tomcat8'
 include 'extensions/geode-modules-session-internal'
 include 'extensions/geode-modules-session'
 include 'extensions/geode-modules-assembly'
-
+include 'geode-protobuf'
 
 if (GradleVersion.current() < GradleVersion.version(minimumGradleVersion)) {
   throw new GradleException('Running with unsupported Gradle Version. Use Gradle Wrapper or with Gradle version >= ' + minimumGradleVersion)
 }
+
+