You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ud...@apache.org on 2017/08/02 16:54:23 UTC

geode git commit: GEODE-3319 - refactor to use protobuf encoding for primitive types. This now closes #661

Repository: geode
Updated Branches:
  refs/heads/develop bf2e0f6e0 -> e5f8e8cb2


GEODE-3319 - refactor to use protobuf encoding for primitive types. This now closes #661


Project: http://git-wip-us.apache.org/repos/asf/geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/geode/commit/e5f8e8cb
Tree: http://git-wip-us.apache.org/repos/asf/geode/tree/e5f8e8cb
Diff: http://git-wip-us.apache.org/repos/asf/geode/diff/e5f8e8cb

Branch: refs/heads/develop
Commit: e5f8e8cb29d81e23ec65aa81d47d4c568fe0b9fa
Parents: bf2e0f6
Author: Hitesh Khamesra <hi...@yahoo.com>
Authored: Wed Jul 26 17:49:27 2017 -0700
Committer: Udo Kohlmeyer <uk...@pivotal.io>
Committed: Wed Aug 2 09:54:00 2017 -0700

----------------------------------------------------------------------
 .../protobuf/EncodingTypeTranslator.java        |  28 +---
 .../operations/PutRequestOperationHandler.java  |   5 +-
 .../utilities/ProtobufPrimitiveTypes.java       |  47 ++++++
 .../protobuf/utilities/ProtobufUtilities.java   | 136 ++++++++++++++--
 .../exception/UnknownProtobufPrimitiveType.java |  21 +++
 .../UnsupportedEncodingTypeException.java       |   4 +
 geode-protobuf/src/main/proto/basicTypes.proto  |  24 +--
 .../geode/protocol/IntegrationJUnitTest.java    | 124 ---------------
 .../RoundTripCacheConnectionJUnitTest.java      |   9 +-
 ...eToSerializationTypeTranslatorJUnitTest.java |  18 ---
 .../GetAllRequestOperationHandlerJUnitTest.java |  28 +---
 ...onNamesRequestOperationHandlerJUnitTest.java |   7 -
 ...tRegionRequestOperationHandlerJUnitTest.java |   3 -
 .../GetRequestOperationHandlerJUnitTest.java    |  44 +++--
 .../PutAllRequestOperationHandlerJUnitTest.java |  25 +--
 .../PutRequestOperationHandlerJUnitTest.java    |  50 +++---
 .../RemoveRequestOperationHandlerJUnitTest.java |  25 +--
 .../utilities/ProtobufUtilitiesJUnitTest.java   | 159 +++++++++++++++++++
 .../ProtobufSerializationServiceJUnitTest.java  | 112 -------------
 19 files changed, 428 insertions(+), 441 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/EncodingTypeTranslator.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/EncodingTypeTranslator.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/EncodingTypeTranslator.java
index 1f65b9e..ec12661 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/EncodingTypeTranslator.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/EncodingTypeTranslator.java
@@ -29,41 +29,15 @@ public abstract class EncodingTypeTranslator {
 
   private static HashMap<Class, BasicTypes.EncodingType> intializeTypeMap() {
     HashMap<Class, BasicTypes.EncodingType> result = new HashMap<>();
-    result.put(Integer.class, BasicTypes.EncodingType.INT);
-    result.put(Byte.class, BasicTypes.EncodingType.BYTE);
-    result.put(Long.class, BasicTypes.EncodingType.LONG);
-    result.put(Float.class, BasicTypes.EncodingType.FLOAT);
-    result.put(Short.class, BasicTypes.EncodingType.SHORT);
-    result.put(byte[].class, BasicTypes.EncodingType.BINARY);
-    result.put(Double.class, BasicTypes.EncodingType.DOUBLE);
-    result.put(String.class, BasicTypes.EncodingType.STRING);
-    result.put(Boolean.class, BasicTypes.EncodingType.BOOLEAN);
+    result.put(PdxInstance.class, BasicTypes.EncodingType.JSON);
     return result;
   }
 
   public static SerializationType getSerializationTypeForEncodingType(
       BasicTypes.EncodingType encodingType) throws UnsupportedEncodingTypeException {
     switch (encodingType) {
-      case INT:
-        return SerializationType.INT;
-      case BYTE:
-        return SerializationType.BYTE;
       case JSON:
         return SerializationType.JSON;
-      case LONG:
-        return SerializationType.LONG;
-      case FLOAT:
-        return SerializationType.FLOAT;
-      case SHORT:
-        return SerializationType.SHORT;
-      case BINARY:
-        return SerializationType.BINARY;
-      case DOUBLE:
-        return SerializationType.DOUBLE;
-      case STRING:
-        return SerializationType.STRING;
-      case BOOLEAN:
-        return SerializationType.BOOLEAN;
       default:
         throw new UnsupportedEncodingTypeException(
             "No serialization type found for protobuf encoding type: " + encodingType);

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandler.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandler.java
index ccfd0c1..ce61c55 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandler.java
@@ -50,10 +50,7 @@ public class PutRequestOperationHandler
         return Success.of(RegionAPI.PutResponse.newBuilder().build());
       } catch (ClassCastException ex) {
         return Failure.of(BasicTypes.ErrorResponse.newBuilder()
-            .setMessage("invalid key or value type for region " + regionName + ",passed key: "
-                + entry.getKey().getEncodingType() + " value: "
-                + entry.getValue().getEncodingType())
-            .build());
+            .setMessage("invalid key or value type for region " + regionName).build());
       }
     } catch (UnsupportedEncodingTypeException ex) {
       return Failure.of(BasicTypes.ErrorResponse.newBuilder().setMessage(ex.getMessage()).build());

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/ProtobufPrimitiveTypes.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/ProtobufPrimitiveTypes.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/ProtobufPrimitiveTypes.java
new file mode 100644
index 0000000..b26de20
--- /dev/null
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/ProtobufPrimitiveTypes.java
@@ -0,0 +1,47 @@
+/*
+ * 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.protobuf.utilities;
+
+import org.apache.geode.protocol.protobuf.utilities.exception.UnknownProtobufPrimitiveType;
+
+public enum ProtobufPrimitiveTypes {
+
+  STRING(String.class),
+  INT(Integer.class),
+  LONG(Long.class),
+  SHORT(Short.class),
+  BYTE(Byte.class),
+  BOOLEAN(Boolean.class),
+  DOUBLE(Double.class),
+  FLOAT(Float.class),
+  BINARY(byte[].class);
+
+  private Class clazzType;
+
+  ProtobufPrimitiveTypes(Class clazz) {
+    this.clazzType = clazz;
+  }
+
+  public static ProtobufPrimitiveTypes valueOf(Class unencodedValueClass)
+      throws UnknownProtobufPrimitiveType {
+    for (ProtobufPrimitiveTypes protobufPrimitiveTypes : values()) {
+      if (protobufPrimitiveTypes.clazzType.equals(unencodedValueClass)) {
+        return protobufPrimitiveTypes;
+      }
+    }
+    throw new UnknownProtobufPrimitiveType(
+        "There is no primitive protobuf type mapping for class:" + unencodedValueClass);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/ProtobufUtilities.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/ProtobufUtilities.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/ProtobufUtilities.java
index 27c141d..fd35803 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/ProtobufUtilities.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/ProtobufUtilities.java
@@ -23,6 +23,7 @@ import org.apache.geode.protocol.protobuf.ClientProtocol;
 import org.apache.geode.protocol.protobuf.EncodingTypeTranslator;
 import org.apache.geode.protocol.protobuf.ProtobufSerializationService;
 import org.apache.geode.protocol.protobuf.RegionAPI;
+import org.apache.geode.protocol.protobuf.utilities.exception.UnknownProtobufPrimitiveType;
 import org.apache.geode.serialization.SerializationService;
 import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
 import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
@@ -39,7 +40,7 @@ import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTy
 public abstract class ProtobufUtilities {
   /**
    * Creates a object containing the type and value encoding of a piece of data
-   * 
+   *
    * @param serializationService - object which knows how to encode objects for the protobuf
    *        protocol {@link ProtobufSerializationService}
    * @param unencodedValue - the value object which is to be encoded
@@ -52,16 +53,24 @@ public abstract class ProtobufUtilities {
   public static BasicTypes.EncodedValue createEncodedValue(
       SerializationService serializationService, Object unencodedValue)
       throws UnsupportedEncodingTypeException, CodecNotRegisteredForTypeException {
-    BasicTypes.EncodingType resultEncodingType =
-        EncodingTypeTranslator.getEncodingTypeForObject(unencodedValue);
-    byte[] encodedValue = serializationService.encode(resultEncodingType, unencodedValue);
-    return BasicTypes.EncodedValue.newBuilder().setEncodingType(resultEncodingType)
-        .setValue(ByteString.copyFrom(encodedValue)).build();
+
+    try {
+      return createPrimitiveEncodedValue(unencodedValue);
+    } catch (UnknownProtobufPrimitiveType e) {
+      BasicTypes.EncodingType resultEncodingType =
+          EncodingTypeTranslator.getEncodingTypeForObject(unencodedValue);
+      byte[] encodedValue = serializationService.encode(resultEncodingType, unencodedValue);
+      BasicTypes.CustomEncodedValue.Builder customEncodedValueBuilder =
+          BasicTypes.CustomEncodedValue.newBuilder().setEncodingType(resultEncodingType)
+              .setValue(ByteString.copyFrom(encodedValue));
+      return BasicTypes.EncodedValue.newBuilder().setCustomEncodedValue(customEncodedValueBuilder)
+          .build();
+    }
   }
 
   /**
    * Creates a protobuf key,value pair from an encoded key and value
-   * 
+   *
    * @param key - an EncodedValue containing the key of the entry
    * @param value - an EncodedValue containing the value of the entry
    * @return a protobuf Entry object containing the passed key and value
@@ -73,7 +82,7 @@ public abstract class ProtobufUtilities {
 
   /**
    * Creates a protobuf key,value pair from unencoded data
-   * 
+   *
    * @param serializationService - object which knows how to encode objects for the protobuf
    *        protocol {@link ProtobufSerializationService}
    * @param unencodedKey - the unencoded key for the entry
@@ -93,7 +102,7 @@ public abstract class ProtobufUtilities {
 
   /**
    * This creates a protobuf message containing a ClientProtocol.Response
-   * 
+   *
    * @param messageHeader - The header for the message
    * @param response - The response for the message
    * @return a protobuf Message containing the above parameters
@@ -106,7 +115,7 @@ public abstract class ProtobufUtilities {
 
   /**
    * This creates a protobuf message containing a ClientProtocol.Request
-   * 
+   *
    * @param messageHeader - The header for the message
    * @param request - The request for the message
    * @return a protobuf Message containing the above parameters
@@ -119,7 +128,7 @@ public abstract class ProtobufUtilities {
 
   /**
    * This creates a protobuf message containing a ClientProtocol.Request
-   * 
+   *
    * @param getAllRequest - The request for the message
    * @return a protobuf Message containing the above parameters
    */
@@ -130,7 +139,7 @@ public abstract class ProtobufUtilities {
 
   /**
    * This builds the MessageHeader for a response which matches an incoming request
-   * 
+   *
    * @param request - The request message that we're responding to.
    * @return the MessageHeader the response to the passed request
    */
@@ -141,7 +150,7 @@ public abstract class ProtobufUtilities {
 
   /**
    * This creates a MessageHeader
-   * 
+   *
    * @param correlationId - An identifier used to correlate requests and responses
    * @return a MessageHeader containing the above parameters
    */
@@ -151,7 +160,7 @@ public abstract class ProtobufUtilities {
 
   /**
    * This will return the object encoded in a protobuf EncodedValue
-   * 
+   *
    * @param serializationService - object which knows how to encode objects for the protobuf
    *        protocol {@link ProtobufSerializationService}
    * @param encodedValue - The value to be decoded
@@ -164,14 +173,25 @@ public abstract class ProtobufUtilities {
   public static Object decodeValue(SerializationService serializationService,
       BasicTypes.EncodedValue encodedValue)
       throws UnsupportedEncodingTypeException, CodecNotRegisteredForTypeException {
-    BasicTypes.EncodingType encoding = encodedValue.getEncodingType();
-    byte[] bytes = encodedValue.getValue().toByteArray();
-    return serializationService.decode(encoding, bytes);
+
+    if (encodedValue.getValueCase() == BasicTypes.EncodedValue.ValueCase.CUSTOMENCODEDVALUE) {
+      BasicTypes.CustomEncodedValue customEncodedValue = encodedValue.getCustomEncodedValue();
+      return serializationService.decode(customEncodedValue.getEncodingType(),
+          customEncodedValue.getValue().toByteArray());
+    } else {
+      try {
+        return getPrimitiveValueFromEncodedValue(encodedValue);
+      } catch (UnknownProtobufPrimitiveType unknownProtobufPrimitiveType) {
+        throw new UnsupportedEncodingTypeException("Unknown primitive type encoding",
+            unknownProtobufPrimitiveType);
+      }
+    }
   }
 
   /**
    * @return a Protobuf BasicTypes.Region message that represents the {@link Region}
    */
+
   public static BasicTypes.Region createRegionMessageFromRegion(Region region) {
     RegionAttributes regionAttributes = region.getAttributes();
     BasicTypes.Region.Builder protoRegionBuilder = BasicTypes.Region.newBuilder();
@@ -201,4 +221,86 @@ public abstract class ProtobufUtilities {
   public static ClientProtocol.Request.Builder createProtobufRequestBuilder() {
     return ClientProtocol.Request.newBuilder();
   }
+
+  /**
+   * This will create an EncodedValue message for a primitive type.
+   *
+   * @param valueToEncode this represents the potential primitive value that needs to be encoded in
+   *        an EncodedValue
+   * @return EncodedValue message with the correct primitive value populated
+   * @throws UnknownProtobufPrimitiveType
+   */
+  static BasicTypes.EncodedValue createPrimitiveEncodedValue(Object valueToEncode)
+      throws UnknownProtobufPrimitiveType {
+    ProtobufPrimitiveTypes protobufPrimitiveTypes =
+        ProtobufPrimitiveTypes.valueOf(valueToEncode.getClass());
+    BasicTypes.EncodedValue.Builder builder = BasicTypes.EncodedValue.newBuilder();
+    switch (protobufPrimitiveTypes) {
+      case INT: {
+        builder.setIntResult((Integer) valueToEncode);
+        break;
+      }
+      case LONG: {
+        builder.setLongResult((Long) valueToEncode);
+        break;
+      }
+      case SHORT: {
+        builder.setShortResult((Short) valueToEncode);
+        break;
+      }
+      case BYTE: {
+        builder.setByteResult((Byte) valueToEncode);
+        break;
+      }
+      case DOUBLE: {
+        builder.setDoubleResult((Double) valueToEncode);
+        break;
+      }
+      case FLOAT: {
+        builder.setFloatResult((Float) valueToEncode);
+        break;
+      }
+      case BINARY: {
+        builder.setBinaryResult(ByteString.copyFrom((byte[]) valueToEncode));
+        break;
+      }
+      case BOOLEAN: {
+        builder.setBooleanResult((Boolean) valueToEncode);
+        break;
+      }
+      case STRING: {
+        builder.setStringResult((String) valueToEncode);
+        break;
+      }
+
+    }
+    return builder.build();
+  }
+
+  static Object getPrimitiveValueFromEncodedValue(BasicTypes.EncodedValue encodedValue)
+      throws UnknownProtobufPrimitiveType {
+    switch (encodedValue.getValueCase()) {
+      case BINARYRESULT:
+        return encodedValue.getBinaryResult().toByteArray();
+      case BOOLEANRESULT:
+        return encodedValue.getBooleanResult();
+      case BYTERESULT:
+        return (byte) encodedValue.getByteResult();
+      case DOUBLERESULT:
+        return encodedValue.getDoubleResult();
+      case FLOATRESULT:
+        return encodedValue.getFloatResult();
+      case INTRESULT:
+        return encodedValue.getIntResult();
+      case LONGRESULT:
+        return encodedValue.getLongResult();
+      case SHORTRESULT:
+        return (short) encodedValue.getShortResult();
+      case STRINGRESULT:
+        return encodedValue.getStringResult();
+      default:
+        throw new UnknownProtobufPrimitiveType(
+            "Unknown primitive type for: " + encodedValue.getValueCase());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/exception/UnknownProtobufPrimitiveType.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/exception/UnknownProtobufPrimitiveType.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/exception/UnknownProtobufPrimitiveType.java
new file mode 100644
index 0000000..675a2f0
--- /dev/null
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/utilities/exception/UnknownProtobufPrimitiveType.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.protocol.protobuf.utilities.exception;
+
+public class UnknownProtobufPrimitiveType extends Exception {
+  public UnknownProtobufPrimitiveType(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/main/java/org/apache/geode/serialization/exception/UnsupportedEncodingTypeException.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/serialization/exception/UnsupportedEncodingTypeException.java b/geode-protobuf/src/main/java/org/apache/geode/serialization/exception/UnsupportedEncodingTypeException.java
index 6d2a032..1056002 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/serialization/exception/UnsupportedEncodingTypeException.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/serialization/exception/UnsupportedEncodingTypeException.java
@@ -21,4 +21,8 @@ public class UnsupportedEncodingTypeException extends Exception {
   public UnsupportedEncodingTypeException(String message) {
     super(message);
   }
+
+  public UnsupportedEncodingTypeException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/main/proto/basicTypes.proto
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/proto/basicTypes.proto b/geode-protobuf/src/main/proto/basicTypes.proto
index a9d07d8..330b53b 100644
--- a/geode-protobuf/src/main/proto/basicTypes.proto
+++ b/geode-protobuf/src/main/proto/basicTypes.proto
@@ -25,21 +25,27 @@ message Entry {
 }
 
 message EncodedValue {
+    oneof value{
+        int32 intResult = 1;
+        int64 longResult = 2;
+        int32 shortResult = 3;
+        int32 byteResult = 4;
+        bool booleanResult = 5;
+        double doubleResult = 6;
+        float floatResult = 7;
+        bytes binaryResult = 8;
+        string stringResult = 9;
+        CustomEncodedValue customEncodedValue = 50;
+    }
+}
+
+message CustomEncodedValue {
     EncodingType encodingType = 1;
     bytes value = 2;
 }
 
 enum EncodingType {
     INVALID = 0;
-    INT = 1;
-    LONG = 2;
-    SHORT = 3;
-    BYTE = 4;
-    BOOLEAN = 5;
-    BINARY = 6;
-    FLOAT = 7;
-    DOUBLE = 8;
-    STRING = 9;
     JSON = 10;
 }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/IntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/IntegrationJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/IntegrationJUnitTest.java
deleted file mode 100644
index 7b8689b..0000000
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/IntegrationJUnitTest.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.protocol;
-
-import static org.mockito.ArgumentMatchers.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-
-import com.google.protobuf.ByteString;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.Region;
-import org.apache.geode.protocol.protobuf.BasicTypes;
-import org.apache.geode.protocol.protobuf.ClientProtocol;
-import org.apache.geode.protocol.protobuf.EncodingTypeTranslator;
-import org.apache.geode.protocol.protobuf.ProtobufStreamProcessor;
-import org.apache.geode.protocol.protobuf.RegionAPI;
-import org.apache.geode.serialization.codec.StringCodec;
-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;
-
-@Category(UnitTest.class)
-public class IntegrationJUnitTest {
-
-  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;
-  private Region regionStub;
-
-  @Before
-  public void setup() throws CodecAlreadyRegisteredForTypeException,
-      UnsupportedEncodingTypeException, CodecNotRegisteredForTypeException {
-    SerializationCodecRegistry serializationCodecRegistry = new SerializationCodecRegistry();
-    stringCodec = (StringCodec) serializationCodecRegistry.getCodecForType(
-        EncodingTypeTranslator.getSerializationTypeForEncodingType(BasicTypes.EncodingType.STRING));
-    regionStub = getRegionStub();
-    cacheStub = getCacheStub(regionStub);
-  }
-
-  @Test
-  public void testGetRequestProcessed() throws Exception {
-    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);
-    verify(regionStub, times(1)).get(TEST_KEY);
-    verify(regionStub, times(1)).get(anyString());
-  }
-
-  private Region getRegionStub() {
-    regionStub = mock(Region.class);
-    when(regionStub.get(TEST_KEY)).thenReturn(TEST_VALUE);
-    return regionStub;
-  }
-
-  private Cache getCacheStub(Region region) {
-    Cache cacheStub = mock(Cache.class);
-    when(cacheStub.getRegion(TEST_REGION)).thenReturn(region);
-    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/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripCacheConnectionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripCacheConnectionJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripCacheConnectionJUnitTest.java
index 5005314..12cc08b 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripCacheConnectionJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripCacheConnectionJUnitTest.java
@@ -265,11 +265,7 @@ public class RoundTripCacheConnectionJUnitTest {
         response.getResponseAPICase());
     RegionAPI.GetResponse getResponse = response.getGetResponse();
 
-    // All the following ways of checking a null response are valid.
     assertFalse(getResponse.hasResult());
-    assertEquals(BasicTypes.EncodingType.INVALID, getResponse.getResult().getEncodingType());
-    assertEquals(null,
-        ProtobufUtilities.decodeValue(serializationService, getResponse.getResult()));
   }
 
   @Test
@@ -342,9 +338,8 @@ public class RoundTripCacheConnectionJUnitTest {
         response.getResponseAPICase());
     RegionAPI.GetResponse getResponse = response.getGetResponse();
     BasicTypes.EncodedValue result = getResponse.getResult();
-    assertEquals(BasicTypes.EncodingType.STRING, result.getEncodingType());
-    assertEquals(expectedValue, new ProtobufSerializationService().decode(result.getEncodingType(),
-        result.getValue().toByteArray()));
+    assertEquals(BasicTypes.EncodedValue.ValueCase.STRINGRESULT, result.getValueCase());
+    assertEquals(expectedValue, result.getStringResult());
   }
 
   private ClientProtocol.Response deserializeResponse(Socket socket,

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/EncodingTypeToSerializationTypeTranslatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/EncodingTypeToSerializationTypeTranslatorJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/EncodingTypeToSerializationTypeTranslatorJUnitTest.java
index a4c6c70..fc1a96d 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/EncodingTypeToSerializationTypeTranslatorJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/EncodingTypeToSerializationTypeTranslatorJUnitTest.java
@@ -28,24 +28,6 @@ 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));
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
index 706eebd..f2e3199 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
@@ -56,12 +56,6 @@ public class GetAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Before
   public void setUp() throws Exception {
     super.setUp();
-    addStringMockEncoding(serializationServiceStub, TEST_KEY1, true, true);
-    addStringMockEncoding(serializationServiceStub, TEST_KEY2, true, true);
-    addStringMockEncoding(serializationServiceStub, TEST_KEY3, true, true);
-    addStringMockEncoding(serializationServiceStub, TEST_VALUE1, true, false);
-    addStringMockEncoding(serializationServiceStub, TEST_VALUE2, true, false);
-    addStringMockEncoding(serializationServiceStub, TEST_VALUE3, true, false);
 
     Region regionStub = mock(Region.class);
     when(regionStub.getAll(new HashSet<Object>() {
@@ -83,18 +77,6 @@ public class GetAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
     stringDecoder = new StringCodec();
   }
 
-  private void addStringMockEncoding(SerializationService mock, String s, boolean add_encoding,
-      boolean add_decoding) throws Exception {
-    if (add_encoding) {
-      when(mock.encode(BasicTypes.EncodingType.STRING, s))
-          .thenReturn(s.getBytes(Charset.forName("UTF-8")));
-    }
-    if (add_decoding) {
-      when(mock.decode(BasicTypes.EncodingType.STRING, s.getBytes(Charset.forName("UTF-8"))))
-          .thenReturn(s);
-    }
-  }
-
   @Test
   public void processReturnsExpectedValuesForValidKeys()
       throws CodecAlreadyRegisteredForTypeException, UnsupportedEncodingTypeException,
@@ -144,11 +126,13 @@ public class GetAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
     Map<String, String> result = new HashMap<>();
     for (BasicTypes.Entry entry : entriesList) {
       BasicTypes.EncodedValue encodedKey = entry.getKey();
-      Assert.assertEquals(BasicTypes.EncodingType.STRING, encodedKey.getEncodingType());
-      String key = stringDecoder.decode(encodedKey.getValue().toByteArray());
+      Assert.assertEquals(BasicTypes.EncodedValue.ValueCase.STRINGRESULT,
+          encodedKey.getValueCase());
+      String key = encodedKey.getStringResult();
       BasicTypes.EncodedValue encodedValue = entry.getValue();
-      Assert.assertEquals(BasicTypes.EncodingType.STRING, encodedValue.getEncodingType());
-      String value = stringDecoder.decode(encodedValue.getValue().toByteArray());
+      Assert.assertEquals(BasicTypes.EncodedValue.ValueCase.STRINGRESULT,
+          encodedValue.getValueCase());
+      String value = encodedValue.getStringResult();
       result.put(key, value);
     }
     return result;

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
index 1ae5eb6..29902e6 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
@@ -49,13 +49,6 @@ public class GetRegionNamesRequestOperationHandlerJUnitTest extends OperationHan
   public void setUp() throws Exception {
     super.setUp();
 
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, TEST_REGION1))
-        .thenReturn(TEST_REGION1.getBytes(Charset.forName("UTF-8")));
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, TEST_REGION2))
-        .thenReturn(TEST_REGION2.getBytes(Charset.forName("UTF-8")));
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, TEST_REGION3))
-        .thenReturn(TEST_REGION3.getBytes(Charset.forName("UTF-8")));
-
     Region<String, String> region1Stub = mock(Region.class);
     when(region1Stub.getName()).thenReturn(TEST_REGION1);
     Region<String, String> region2Stub = mock(Region.class);

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
index 7423326..1392aec 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
@@ -51,9 +51,6 @@ public class GetRegionRequestOperationHandlerJUnitTest extends OperationHandlerJ
   public void setUp() throws Exception {
     super.setUp();
 
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, TEST_REGION1))
-        .thenReturn(TEST_REGION1.getBytes(Charset.forName("UTF-8")));
-
     region1Stub = mock(Region.class);
     when(region1Stub.getName()).thenReturn(TEST_REGION1);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
index 95a3893..6710f1a 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
@@ -14,11 +14,17 @@
  */
 package org.apache.geode.protocol.protobuf.operations;
 
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.ArgumentMatchers.anyObject;
+import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.io.UnsupportedEncodingException;
 import java.nio.charset.Charset;
 
+import com.google.protobuf.ByteString;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -37,6 +43,7 @@ import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredF
 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.mockito.internal.matchers.Any;
 
 @Category(UnitTest.class)
 public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTest {
@@ -52,21 +59,6 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   public void setUp() throws Exception {
     super.setUp();
 
-    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")));
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, TEST_KEY))
-        .thenReturn(TEST_KEY.getBytes(Charset.forName("UTF-8")));
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, MISSING_KEY))
-        .thenReturn(MISSING_KEY.getBytes(Charset.forName("UTF-8")));
-    when(serializationServiceStub.decode(BasicTypes.EncodingType.STRING,
-        MISSING_KEY.getBytes(Charset.forName("UTF-8")))).thenReturn(MISSING_KEY);
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, NULLED_KEY))
-        .thenReturn(NULLED_KEY.getBytes(Charset.forName("UTF-8")));
-    when(serializationServiceStub.decode(BasicTypes.EncodingType.STRING,
-        NULLED_KEY.getBytes(Charset.forName("UTF-8")))).thenReturn(NULLED_KEY);
-
     Region regionStub = mock(Region.class);
     when(regionStub.get(TEST_KEY)).thenReturn(TEST_VALUE);
     when(regionStub.get(MISSING_KEY)).thenReturn(null);
@@ -89,10 +81,9 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
         operationHandler.process(serializationServiceStub, getRequest, cacheStub);
 
     Assert.assertTrue(result instanceof Success);
-    Assert.assertEquals(BasicTypes.EncodingType.STRING,
-        result.getMessage().getResult().getEncodingType());
-    String actualValue =
-        stringDecoder.decode(result.getMessage().getResult().getValue().toByteArray());
+    Assert.assertEquals(BasicTypes.EncodedValue.ValueCase.STRINGRESULT,
+        result.getMessage().getResult().getValueCase());
+    String actualValue = result.getMessage().getResult().getStringResult();
     Assert.assertEquals(TEST_VALUE, actualValue);
   }
 
@@ -133,13 +124,18 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   @Test
   public void processReturnsErrorWhenUnableToDecodeRequest()
       throws CodecAlreadyRegisteredForTypeException, UnsupportedEncodingTypeException,
-      CodecNotRegisteredForTypeException {
+      CodecNotRegisteredForTypeException, UnsupportedEncodingException {
     CodecNotRegisteredForTypeException exception =
         new CodecNotRegisteredForTypeException("error finding codec for type");
-    when(serializationServiceStub.decode(BasicTypes.EncodingType.STRING,
-        TEST_KEY.getBytes(Charset.forName("UTF-8")))).thenThrow(exception);
-
-    RegionAPI.GetRequest getRequest = generateTestRequest(false, false, false);
+    when(serializationServiceStub.decode(any(), any())).thenThrow(exception);
+
+    ByteString byteString = ByteString.copyFrom("{\"someKey\":\"someValue\"}", "UTF-8");
+    BasicTypes.CustomEncodedValue.Builder customEncodedValueBuilder = BasicTypes.CustomEncodedValue
+        .newBuilder().setEncodingType(BasicTypes.EncodingType.JSON).setValue(byteString);
+    BasicTypes.EncodedValue encodedKey = BasicTypes.EncodedValue.newBuilder()
+        .setCustomEncodedValue(customEncodedValueBuilder).build();
+    RegionAPI.GetRequest getRequest =
+        ProtobufRequestUtilities.createGetRequest(TEST_REGION, encodedKey).getGetRequest();
     Result<RegionAPI.GetResponse> response =
         operationHandler.process(serializationServiceStub, getRequest, cacheStub);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
index 33d21e7..0fde9e5 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
@@ -61,20 +61,6 @@ public class PutAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Before
   public void setUp() throws Exception {
     super.setUp();
-
-    addStringStubEncoding(serializationServiceStub, TEST_KEY1);
-    addStringStubEncoding(serializationServiceStub, TEST_KEY2);
-    addStringStubEncoding(serializationServiceStub, TEST_KEY3);
-    addStringStubEncoding(serializationServiceStub, TEST_INVALID_KEY);
-    addStringStubEncoding(serializationServiceStub, TEST_VALUE1);
-    addStringStubEncoding(serializationServiceStub, TEST_VALUE2);
-    addStringStubEncoding(serializationServiceStub, TEST_VALUE3);
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.INT, TEST_INVALID_VALUE))
-        .thenReturn(ByteBuffer.allocate(Integer.BYTES).putInt(TEST_INVALID_VALUE).array());
-    when(serializationServiceStub.decode(BasicTypes.EncodingType.INT,
-        ByteBuffer.allocate(Integer.BYTES).putInt(TEST_INVALID_VALUE).array()))
-            .thenReturn(TEST_INVALID_VALUE);
-
     regionMock = mock(Region.class);
     when(regionMock.put(TEST_INVALID_KEY, TEST_INVALID_VALUE))
         .thenThrow(new ClassCastException(EXCEPTION_TEXT));
@@ -82,13 +68,6 @@ public class PutAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
     when(cacheStub.getRegion(TEST_REGION)).thenReturn(regionMock);
   }
 
-  private void addStringStubEncoding(SerializationService stub, String s) throws Exception {
-    when(stub.encode(BasicTypes.EncodingType.STRING, s))
-        .thenReturn(s.getBytes(Charset.forName("UTF-8")));
-    when(stub.decode(BasicTypes.EncodingType.STRING, s.getBytes(Charset.forName("UTF-8"))))
-        .thenReturn(s);
-  }
-
   @Test
   public void processInsertsMultipleValidEntriesInCache() throws UnsupportedEncodingTypeException,
       CodecNotRegisteredForTypeException, CodecAlreadyRegisteredForTypeException {
@@ -119,8 +98,8 @@ public class PutAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
     RegionAPI.PutAllResponse putAllResponse = result.getMessage();
     assertEquals(1, putAllResponse.getFailedKeysCount());
     BasicTypes.KeyedErrorResponse error = putAllResponse.getFailedKeys(0);
-    assertEquals(TEST_INVALID_KEY, serializationServiceStub.decode(error.getKey().getEncodingType(),
-        error.getKey().getValue().toByteArray()));
+    assertEquals(TEST_INVALID_KEY,
+        ProtobufUtilities.decodeValue(serializationServiceStub, error.getKey()));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
index 8c1652b..144f5b8 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.protocol.protobuf.operations;
 
+import com.google.protobuf.ByteString;
 import org.apache.geode.cache.Region;
 import org.apache.geode.protocol.protobuf.BasicTypes;
 import org.apache.geode.protocol.protobuf.Failure;
@@ -32,7 +33,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.nio.charset.Charset;
+import java.io.UnsupportedEncodingException;
 
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.anyString;
@@ -52,15 +53,6 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   public void setUp() throws Exception {
     super.setUp();
 
-    when(serializationServiceStub.decode(BasicTypes.EncodingType.STRING,
-        TEST_KEY.getBytes(Charset.forName("UTF-8")))).thenReturn(TEST_KEY);
-    when(serializationServiceStub.decode(BasicTypes.EncodingType.STRING,
-        TEST_VALUE.getBytes(Charset.forName("UTF-8")))).thenReturn(TEST_VALUE);
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, TEST_KEY))
-        .thenReturn(TEST_KEY.getBytes(Charset.forName("UTF-8")));
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, TEST_VALUE))
-        .thenReturn(TEST_VALUE.getBytes(Charset.forName("UTF-8")));
-
     regionMock = mock(Region.class);
     when(regionMock.put(TEST_KEY, TEST_VALUE)).thenReturn(1);
 
@@ -81,38 +73,32 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   }
 
   @Test
-  public void test_invalidEncodingType() throws CodecAlreadyRegisteredForTypeException,
-      UnsupportedEncodingTypeException, CodecNotRegisteredForTypeException {
+  public void test_invalidEncodingType()
+      throws CodecAlreadyRegisteredForTypeException, UnsupportedEncodingTypeException,
+      CodecNotRegisteredForTypeException, UnsupportedEncodingException {
     String exceptionText = "unsupported type!";
     UnsupportedEncodingTypeException exception =
         new UnsupportedEncodingTypeException(exceptionText);
-    when(serializationServiceStub.decode(BasicTypes.EncodingType.STRING,
-        TEST_KEY.getBytes(Charset.forName("UTF-8")))).thenThrow(exception);
-    PutRequestOperationHandler operationHandler = new PutRequestOperationHandler();
-
-    Result<RegionAPI.PutResponse> result =
-        operationHandler.process(serializationServiceStub, generateTestRequest(), cacheStub);
+    when(serializationServiceStub.decode(any(), any())).thenThrow(exception);
 
-    Assert.assertTrue(result instanceof Failure);
-    org.junit.Assert.assertEquals(exceptionText, result.getErrorMessage().getMessage());
-  }
+    ByteString byteString = ByteString.copyFrom("{\"someKey\":\"someValue\"}", "UTF-8");
+    BasicTypes.CustomEncodedValue.Builder customEncodedValueBuilder = BasicTypes.CustomEncodedValue
+        .newBuilder().setEncodingType(BasicTypes.EncodingType.JSON).setValue(byteString);
+    BasicTypes.EncodedValue encodedKey = BasicTypes.EncodedValue.newBuilder()
+        .setCustomEncodedValue(customEncodedValueBuilder).build();
 
-  @Test
-  public void test_codecNotRegistered() throws CodecAlreadyRegisteredForTypeException,
-      UnsupportedEncodingTypeException, CodecNotRegisteredForTypeException {
-    String exceptionMessage = "error finding codec for type";
-    CodecNotRegisteredForTypeException exception =
-        new CodecNotRegisteredForTypeException(exceptionMessage);
-    when(serializationServiceStub.decode(BasicTypes.EncodingType.STRING,
-        TEST_KEY.getBytes(Charset.forName("UTF-8")))).thenThrow(exception);
     PutRequestOperationHandler operationHandler = new PutRequestOperationHandler();
 
+    BasicTypes.EncodedValue testValue =
+        ProtobufUtilities.createEncodedValue(serializationServiceStub, TEST_VALUE);
+    BasicTypes.Entry testEntry = ProtobufUtilities.createEntry(encodedKey, testValue);
+    RegionAPI.PutRequest putRequest =
+        ProtobufRequestUtilities.createPutRequest(TEST_REGION, testEntry).getPutRequest();
     Result<RegionAPI.PutResponse> result =
-        operationHandler.process(serializationServiceStub, generateTestRequest(), cacheStub);
+        operationHandler.process(serializationServiceStub, putRequest, cacheStub);
 
     Assert.assertTrue(result instanceof Failure);
-    org.junit.Assert.assertThat(result.getErrorMessage().getMessage(),
-        CoreMatchers.containsString(exceptionMessage));
+    org.junit.Assert.assertEquals(exceptionText, result.getErrorMessage().getMessage());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
index cf434c2..36982a1 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
@@ -15,12 +15,15 @@
 package org.apache.geode.protocol.protobuf.operations;
 
 import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.io.UnsupportedEncodingException;
 import java.nio.charset.Charset;
 
+import com.google.protobuf.ByteString;
 import org.hamcrest.CoreMatchers;
 import org.junit.Before;
 import org.junit.Test;
@@ -54,14 +57,6 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Before
   public void setUp() throws Exception {
     super.setUp();
-    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")));
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, TEST_KEY))
-        .thenReturn(TEST_KEY.getBytes(Charset.forName("UTF-8")));
-    when(serializationServiceStub.encode(BasicTypes.EncodingType.STRING, MISSING_KEY))
-        .thenReturn(MISSING_KEY.getBytes(Charset.forName("UTF-8")));
 
     regionStub = mock(Region.class);
     when(regionStub.remove(TEST_KEY)).thenReturn(TEST_VALUE);
@@ -113,13 +108,19 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Test
   public void processReturnsErrorWhenUnableToDecodeRequest()
       throws CodecAlreadyRegisteredForTypeException, UnsupportedEncodingTypeException,
-      CodecNotRegisteredForTypeException {
+      CodecNotRegisteredForTypeException, UnsupportedEncodingException {
     CodecNotRegisteredForTypeException exception =
         new CodecNotRegisteredForTypeException("error finding codec for type");
-    when(serializationServiceStub.decode(BasicTypes.EncodingType.STRING,
-        TEST_KEY.getBytes(Charset.forName("UTF-8")))).thenThrow(exception);
+    when(serializationServiceStub.decode(any(), any())).thenThrow(exception);
 
-    RegionAPI.RemoveRequest removeRequest = generateTestRequest(false, false).getRemoveRequest();
+    ByteString byteString = ByteString.copyFrom("{\"someKey\":\"someValue\"}", "UTF-8");
+    BasicTypes.CustomEncodedValue.Builder customEncodedValueBuilder = BasicTypes.CustomEncodedValue
+        .newBuilder().setEncodingType(BasicTypes.EncodingType.JSON).setValue(byteString);
+    BasicTypes.EncodedValue encodedKey = BasicTypes.EncodedValue.newBuilder()
+        .setCustomEncodedValue(customEncodedValueBuilder).build();
+
+    RegionAPI.RemoveRequest removeRequest =
+        ProtobufRequestUtilities.createRemoveRequest(TEST_REGION, encodedKey).getRemoveRequest();;
     Result<RegionAPI.RemoveResponse> result =
         operationHandler.process(serializationServiceStub, removeRequest, cacheStub);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/e5f8e8cb/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/utilities/ProtobufUtilitiesJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/utilities/ProtobufUtilitiesJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/utilities/ProtobufUtilitiesJUnitTest.java
new file mode 100644
index 0000000..46597a5
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/utilities/ProtobufUtilitiesJUnitTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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.protobuf.utilities;
+
+import com.google.protobuf.ByteString;
+import org.apache.geode.protocol.protobuf.BasicTypes;
+import org.apache.geode.protocol.protobuf.utilities.exception.UnknownProtobufPrimitiveType;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.charset.Charset;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+@Category(UnitTest.class)
+public class ProtobufUtilitiesJUnitTest {
+  @Test
+  public void getIntPrimitiveFromEncodedValue() throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue.Builder builder = BasicTypes.EncodedValue.newBuilder();
+    BasicTypes.EncodedValue encodedValue = builder.setIntResult(1).build();
+    assertEquals(1, ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+  }
+
+  @Test
+  public void getLongPrimitiveFromEncodedValue() throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue encodedValue =
+        BasicTypes.EncodedValue.newBuilder().setLongResult(1).build();
+    assertEquals(1l, ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+  }
+
+  @Test
+  public void getShortPrimitiveFromEncodedValue() throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue encodedValue =
+        BasicTypes.EncodedValue.newBuilder().setShortResult(1).build();
+    assertEquals((short) 1, ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+  }
+
+  @Test
+  public void getBytePrimitiveFromEncodedValue() throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue encodedValue =
+        BasicTypes.EncodedValue.newBuilder().setByteResult(1).build();
+    assertEquals((byte) 1, ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+  }
+
+  @Test
+  public void getBooleanPrimitiveFromEncodedValue() throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue encodedValue =
+        BasicTypes.EncodedValue.newBuilder().setBooleanResult(true).build();
+    assertEquals(true, ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+  }
+
+  @Test
+  public void getDoublePrimitiveFromEncodedValue() throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue encodedValue =
+        BasicTypes.EncodedValue.newBuilder().setDoubleResult(1.0).build();
+    assertEquals(1.0, ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+  }
+
+  @Test
+  public void getFloatPrimitiveFromEncodedValue() throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue encodedValue =
+        BasicTypes.EncodedValue.newBuilder().setFloatResult(1).build();
+    assertEquals(1f, ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+  }
+
+  @Test
+  public void getByteArrayPrimitiveFromEncodedValue() throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue encodedValue = BasicTypes.EncodedValue.newBuilder()
+        .setBinaryResult(ByteString.copyFrom("SomeBinary".getBytes())).build();
+    assertArrayEquals("SomeBinary".getBytes(Charset.forName("UTF-8")),
+        (byte[]) ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+  }
+
+  @Test
+  public void getStringPrimitiveFromEncodedValue() throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue encodedValue =
+        BasicTypes.EncodedValue.newBuilder().setStringResult("SomeString").build();
+    assertEquals("SomeString", ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+  }
+
+  @Test
+  public void doesAIntValueSuccessfullyEncodeIntoPrimitiveEncodedValues()
+      throws UnknownProtobufPrimitiveType {
+    createAndVerifyEncodedValue(1);
+  }
+
+  @Test
+  public void doesALongValueSuccessfullyEncodeIntoPrimitiveEncodedValues()
+      throws UnknownProtobufPrimitiveType {
+    createAndVerifyEncodedValue(1l);
+  }
+
+  @Test
+  public void doesAShortValueSuccessfullyEncodeIntoPrimitiveEncodedValues()
+      throws UnknownProtobufPrimitiveType {
+    createAndVerifyEncodedValue((short) 1);
+  }
+
+  @Test
+  public void doesAByteValueSuccessfullyEncodeIntoPrimitiveEncodedValues()
+      throws UnknownProtobufPrimitiveType {
+    createAndVerifyEncodedValue((byte) 1);
+  }
+
+  @Test
+  public void doesABooleanValueSuccessfullyEncodeIntoPrimitiveEncodedValues()
+      throws UnknownProtobufPrimitiveType {
+    createAndVerifyEncodedValue(true);
+  }
+
+  @Test
+  public void doesADoubleValueSuccessfullyEncodeIntoPrimitiveEncodedValues()
+      throws UnknownProtobufPrimitiveType {
+    createAndVerifyEncodedValue((double) 1);
+  }
+
+  @Test
+  public void doesAFloatValueSuccessfullyEncodeIntoPrimitiveEncodedValues()
+      throws UnknownProtobufPrimitiveType {
+    createAndVerifyEncodedValue((float) 1);
+  }
+
+  @Test
+  public void doesABinaryValueSuccessfullyEncodeIntoPrimitiveEncodedValues()
+      throws UnknownProtobufPrimitiveType {
+    createAndVerifyEncodedValue("Some Text to Binary".getBytes());
+  }
+
+  @Test
+  public void doesAStringValueSuccessfullyEncodeIntoPrimitiveEncodedValues()
+      throws UnknownProtobufPrimitiveType {
+    createAndVerifyEncodedValue("Some String text to test");
+  }
+
+  private <T> void createAndVerifyEncodedValue(T testObj) throws UnknownProtobufPrimitiveType {
+    BasicTypes.EncodedValue encodedValue = ProtobufUtilities.createPrimitiveEncodedValue(testObj);
+    if (testObj instanceof byte[]) {
+      assertArrayEquals((byte[]) testObj,
+          (byte[]) ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+    } else {
+      assertEquals(testObj, ProtobufUtilities.getPrimitiveValueFromEncodedValue(encodedValue));
+    }
+  }
+}

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