You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by mz...@apache.org on 2019/07/31 03:24:55 UTC

[mesos] branch master updated (00bb0b6 -> 8175453)

This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from 00bb0b6  Added documentation for GET_OPERATIONS calls.
     new 8eda408  Fixed non-standard mapping for protobuf map fields in jsonify.
     new fecbaf9  Improved a test for jsonify map fields mapping.
     new 8175453  Updated quota related endpoints to return quota configurations.

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 3rdparty/stout/include/stout/protobuf.hpp | 184 +++++++++++++++++++++++-------
 3rdparty/stout/tests/protobuf_tests.cpp   | 126 ++++++++++----------
 src/master/quota_handler.cpp              |  15 +++
 src/tests/master_quota_tests.cpp          | 136 +++++++++++++++++++++-
 4 files changed, 359 insertions(+), 102 deletions(-)


[mesos] 03/03: Updated quota related endpoints to return quota configurations.

Posted by mz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 817545318da364efdff7c9c3f888d0d7aa94da23
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Tue Jul 30 18:48:32 2019 -0700

    Updated quota related endpoints to return quota configurations.
    
    Added quota configuration information (that includes both
    guarantees and limits) in V1 GET_QUOTA call and V0 GET "/quota".
    
    To keep backwards compatibility, the infos field which only
    includes the guarantees are continue to be filled. An additional
    field configs was added.
    
    Also extended an existing test to cover the changes in
    the endpoints.
    
    Review: https://reviews.apache.org/r/71159
---
 src/master/quota_handler.cpp     |  15 +++++
 src/tests/master_quota_tests.cpp | 136 ++++++++++++++++++++++++++++++++++++++-
 2 files changed, 150 insertions(+), 1 deletion(-)

diff --git a/src/master/quota_handler.cpp b/src/master/quota_handler.cpp
index 15b02b0..f28eb27 100644
--- a/src/master/quota_handler.cpp
+++ b/src/master/quota_handler.cpp
@@ -409,6 +409,21 @@ Future<QuotaStatus> Master::QuotaHandler::_status(
             }
             return info;
           }();
+
+          *status.add_configs() = [&quotaIt]() {
+            QuotaConfig config;
+            config.set_role(quotaIt->first);
+
+            foreach (auto& quantity, quotaIt->second.guarantees) {
+              (*config.mutable_guarantees())[quantity.first] = quantity.second;
+            }
+
+            foreach (auto& limit, quotaIt->second.limits) {
+              (*config.mutable_limits())[limit.first] = limit.second;
+            }
+
+            return config;
+          }();
         }
       }
 
diff --git a/src/tests/master_quota_tests.cpp b/src/tests/master_quota_tests.cpp
index 02b1e8d..b5fe235 100644
--- a/src/tests/master_quota_tests.cpp
+++ b/src/tests/master_quota_tests.cpp
@@ -140,6 +140,15 @@ static string createUpdateQuotaRequestBody(
 }
 
 
+static string createGetQuotaRequestBody()
+{
+  mesos::master::Call call;
+  call.set_type(mesos::master::Call::GET_QUOTA);
+
+  return stringify(JSON::protobuf(call));
+}
+
+
 // Quota tests that are allocator-agnostic (i.e. we expect every
 // allocator to implement basic quota guarantees) are in this
 // file. All tests are split into logical groups:
@@ -211,7 +220,7 @@ protected:
 };
 
 
-TEST_F(MasterQuotaTest, UpdateQuota)
+TEST_F(MasterQuotaTest, UpdateAndGetQuota)
 {
   TestAllocator<> allocator;
   EXPECT_CALL(allocator, initialize(_, _, _));
@@ -233,6 +242,7 @@ TEST_F(MasterQuotaTest, UpdateQuota)
 
   AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
 
+  // Verify "/roles".
   response = process::http::get(
       master.get()->pid,
       "roles",
@@ -275,6 +285,130 @@ TEST_F(MasterQuotaTest, UpdateQuota)
 
   EXPECT_EQ(*expected, *parse) << "expected " << stringify(*expected)
                                << " vs actual " << stringify(*parse);
+
+  // Verify `GET_QUOTA`.
+  response = process::http::post(
+      master.get()->pid, "/api/v1", headers, createGetQuotaRequestBody());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+
+  expected = JSON::parse(
+      "{"
+      "  \"type\": \"GET_QUOTA\","
+      "  \"get_quota\": {"
+      "  \"status\": {"
+      "   \"infos\": ["
+      "      {"
+      "        \"role\": \"role1\","
+      "        \"guarantee\": ["
+      "           {"
+      "             \"name\": \"cpus\","
+      "             \"type\": \"SCALAR\","
+      "             \"scalar\": {"
+      "               \"value\": 1"
+      "             }"
+      "           },"
+      "           {"
+      "             \"name\": \"mem\","
+      "             \"type\": \"SCALAR\","
+      "             \"scalar\": {"
+      "               \"value\": 1024"
+      "             }"
+      "           }"
+      "        ]"
+      "      }"
+      "    ],"
+      "    \"configs\": ["
+      "      {"
+      "        \"role\": \"role1\","
+      "        \"guarantees\": {"
+      "          \"mem\": {"
+      "            \"value\": 1024"
+      "          },"
+      "          \"cpus\": {"
+      "            \"value\": 1"
+      "          }"
+      "        },"
+      "        \"limits\": {"
+      "          \"mem\": {"
+      "            \"value\": 2048"
+      "          },"
+      "          \"cpus\": {"
+      "            \"value\": 2"
+      "          }"
+      "        }"
+      "      }"
+      "    ]"
+      "   }"
+      "  }"
+      "}");
+  ASSERT_SOME(expected);
+
+  Try<JSON::Value> actual = JSON::parse(response->body);
+  ASSERT_SOME(actual);
+
+  EXPECT_EQ(*expected, *(actual));
+
+  // Verify get "/quota".
+  response = process::http::get(
+      master.get()->pid,
+      "quota",
+      None(),
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+
+  expected = JSON::parse(
+      "{"
+      " \"infos\": ["
+      "    {"
+      "      \"role\": \"role1\","
+      "      \"guarantee\": ["
+      "         {"
+      "           \"name\": \"cpus\","
+      "           \"type\": \"SCALAR\","
+      "           \"scalar\": {"
+      "             \"value\": 1"
+      "           }"
+      "         },"
+      "         {"
+      "           \"name\": \"mem\","
+      "           \"type\": \"SCALAR\","
+      "           \"scalar\": {"
+      "             \"value\": 1024"
+      "           }"
+      "         }"
+      "      ]"
+      "    }"
+      "  ],"
+      "  \"configs\": ["
+      "    {"
+      "      \"role\": \"role1\","
+      "      \"guarantees\": {"
+      "        \"mem\": {"
+      "          \"value\": 1024"
+      "        },"
+      "        \"cpus\": {"
+      "          \"value\": 1"
+      "        }"
+      "      },"
+      "      \"limits\": {"
+      "        \"mem\": {"
+      "          \"value\": 2048"
+      "        },"
+      "        \"cpus\": {"
+      "          \"value\": 2"
+      "        }"
+      "      }"
+      "    }"
+      "  ]"
+      "}");
+  ASSERT_SOME(expected);
+
+  actual = JSON::parse(response->body);
+  ASSERT_SOME(actual);
+
+  EXPECT_EQ(*expected, *(actual));
 }
 
 


[mesos] 01/03: Fixed non-standard mapping for protobuf map fields in jsonify.

Posted by mz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 8eda408d8c8887a195f3e2eab6cf7d2153d8d193
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Mon Jul 29 14:24:32 2019 -0700

    Fixed non-standard mapping for protobuf map fields in jsonify.
    
    Before this patch jsonify treats protobuf Map as a regular
    repeated field. This means a Map with schema:
    
    message QuotaConfig {
      required string role = 1;
    
      map<string, Value.Scalar> guarantees = 2;
      map<string, Value.Scalar> limits = 3;
    }
    
    may be jsonify to an JSON array:
    
    {
      "configs": [
        {
          "role": "role1",
          "guarantees": [
            {
              "key": "cpus",
              "value": {
                "value": 1
              }
            },
            {
              "key": "mem",
              "value": {
                "value": 512
              }
            }
          ]
        }
      ]
    }
    
    Per standard proto3 JSON mapping, the Map type should be mapped
    to an JSON object, i.e.
    
    {
      "configs": [
        {
          "role": "role1",
          "guarantees": {
            "cpus": 1,
            "mem": 512
          }
        }
      ]
    }
    
    This patch made jsonify support for such mapping.
    
    Currently, there is no egress of map fields in the code base,
    so this presents no external visible change.
    
    Review: https://reviews.apache.org/r/71158
---
 3rdparty/stout/include/stout/protobuf.hpp | 184 +++++++++++++++++++++++-------
 1 file changed, 142 insertions(+), 42 deletions(-)

diff --git a/3rdparty/stout/include/stout/protobuf.hpp b/3rdparty/stout/include/stout/protobuf.hpp
index 4b3db7e..fcd91d5 100644
--- a/3rdparty/stout/include/stout/protobuf.hpp
+++ b/3rdparty/stout/include/stout/protobuf.hpp
@@ -29,6 +29,7 @@
 #include <google/protobuf/descriptor.h>
 #include <google/protobuf/descriptor.pb.h>
 #include <google/protobuf/message.h>
+#include <google/protobuf/reflection.h>
 #include <google/protobuf/repeated_field.h>
 
 #include <google/protobuf/io/zero_copy_stream_impl.h>
@@ -809,6 +810,9 @@ struct Protobuf : Representation<google::protobuf::Message>
 // `json` function for protobuf messages. Refer to `jsonify.hpp` for details.
 // TODO(mpark): This currently uses the default value for optional fields
 // that are not deprecated, but we may want to revisit this decision.
+//
+// TODO(mzhu): Use protobuf built-in JSON mapping utilities in favor of
+// the reflection APIs. See MESOS-9896.
 inline void json(ObjectWriter* writer, const Protobuf& protobuf)
 {
   using google::protobuf::FieldDescriptor;
@@ -827,7 +831,7 @@ inline void json(ObjectWriter* writer, const Protobuf& protobuf)
   fields.reserve(fieldCount);
   for (int i = 0; i < fieldCount; ++i) {
     const FieldDescriptor* field = descriptor->field(i);
-    if (field->is_repeated()) {
+    if (field->is_repeated()) { // Repeated or Map.
       if (reflection->FieldSize(message, field) > 0) {
         // Has repeated field with members, output as JSON.
         fields.push_back(field);
@@ -841,7 +845,7 @@ inline void json(ObjectWriter* writer, const Protobuf& protobuf)
   }
 
   foreach (const FieldDescriptor* field, fields) {
-    if (field->is_repeated()) {
+    if (field->is_repeated() && !field->is_map()) {
       writer->field(
           field->name(),
           [&field, &reflection, &message](JSON::ArrayWriter* writer) {
@@ -896,46 +900,142 @@ inline void json(ObjectWriter* writer, const Protobuf& protobuf)
               }
             }
           });
-    } else {
-      switch (field->cpp_type()) {
-        case FieldDescriptor::CPPTYPE_BOOL:
-          writer->field(field->name(), reflection->GetBool(message, field));
-          break;
-        case FieldDescriptor::CPPTYPE_INT32:
-          writer->field(field->name(), reflection->GetInt32(message, field));
-          break;
-        case FieldDescriptor::CPPTYPE_INT64:
-          writer->field(field->name(), reflection->GetInt64(message, field));
-          break;
-        case FieldDescriptor::CPPTYPE_UINT32:
-          writer->field(field->name(), reflection->GetUInt32(message, field));
-          break;
-        case FieldDescriptor::CPPTYPE_UINT64:
-          writer->field(field->name(), reflection->GetUInt64(message, field));
-          break;
-        case FieldDescriptor::CPPTYPE_FLOAT:
-          writer->field(field->name(), reflection->GetFloat(message, field));
-          break;
-        case FieldDescriptor::CPPTYPE_DOUBLE:
-          writer->field(field->name(), reflection->GetDouble(message, field));
-          break;
-        case FieldDescriptor::CPPTYPE_MESSAGE:
-          writer->field(
-              field->name(), Protobuf(reflection->GetMessage(message, field)));
-          break;
-        case FieldDescriptor::CPPTYPE_ENUM:
-          writer->field(
-              field->name(), reflection->GetEnum(message, field)->name());
-          break;
-        case FieldDescriptor::CPPTYPE_STRING:
-          const std::string& s = reflection->GetStringReference(
-              message, field, nullptr);
-          if (field->type() == FieldDescriptor::TYPE_BYTES) {
-            writer->field(field->name(), base64::encode(s));
-          } else {
-            writer->field(field->name(), s);
-          }
-          break;
+    } else { // field->is_map() || !field->is_repeated()
+      auto writeField = [&writer](
+                            const std::string& fieldName,
+                            const google::protobuf::Reflection* reflection,
+                            const google::protobuf::Message& message,
+                            const FieldDescriptor* field) {
+        switch (field->cpp_type()) {
+          case FieldDescriptor::CPPTYPE_BOOL:
+            writer->field(fieldName, reflection->GetBool(message, field));
+            break;
+          case FieldDescriptor::CPPTYPE_INT32:
+            writer->field(fieldName, reflection->GetInt32(message, field));
+            break;
+          case FieldDescriptor::CPPTYPE_INT64:
+            writer->field(fieldName, reflection->GetInt64(message, field));
+            break;
+          case FieldDescriptor::CPPTYPE_UINT32:
+            writer->field(fieldName, reflection->GetUInt32(message, field));
+            break;
+          case FieldDescriptor::CPPTYPE_UINT64:
+            writer->field(fieldName, reflection->GetUInt64(message, field));
+            break;
+          case FieldDescriptor::CPPTYPE_FLOAT:
+            writer->field(fieldName, reflection->GetFloat(message, field));
+            break;
+          case FieldDescriptor::CPPTYPE_DOUBLE:
+            writer->field(fieldName, reflection->GetDouble(message, field));
+            break;
+          case FieldDescriptor::CPPTYPE_MESSAGE:
+            writer->field(
+                fieldName, Protobuf(reflection->GetMessage(message, field)));
+            break;
+          case FieldDescriptor::CPPTYPE_ENUM:
+            writer->field(
+                fieldName, reflection->GetEnum(message, field)->name());
+            break;
+          case FieldDescriptor::CPPTYPE_STRING:
+            const std::string& s =
+              reflection->GetStringReference(message, field, nullptr);
+            if (field->type() == FieldDescriptor::TYPE_BYTES) {
+              writer->field(fieldName, base64::encode(s));
+            } else {
+              writer->field(fieldName, s);
+            }
+            break;
+        }
+      };
+
+      if (!field->is_repeated()) { // Singular field.
+        writeField(field->name(), reflection, message, field);
+      } else { // Map field.
+        CHECK(field->is_map());
+        writer->field(
+            field->name(),
+            [&field, &reflection, &message, &writeField](
+                JSON::ObjectWriter* writer) {
+              foreach (
+                  const auto& mapEntry,
+                  reflection->GetRepeatedFieldRef<google::protobuf::Message>(
+                      message, field)) {
+                const google::protobuf::Descriptor* mapEntryDescriptor =
+                  mapEntry.GetDescriptor();
+                const google::protobuf::Reflection* mapEntryReflection =
+                  mapEntry.GetReflection();
+
+                // Map entry must contain exactly two fields: `key` and `value`.
+                CHECK_EQ(mapEntryDescriptor->field_count(), 2);
+
+                const FieldDescriptor* keyField = mapEntryDescriptor->field(0);
+                const FieldDescriptor* valueField =
+                  mapEntryDescriptor->field(1);
+
+                switch (keyField->cpp_type()) {
+                  case FieldDescriptor::CPPTYPE_BOOL:
+                    writeField(
+                        jsonify(
+                            mapEntryReflection->GetBool(mapEntry, keyField)),
+                        mapEntryReflection,
+                        mapEntry,
+                        valueField);
+                    break;
+                  case FieldDescriptor::CPPTYPE_INT32:
+                    writeField(
+                        jsonify(
+                            mapEntryReflection->GetInt32(mapEntry, keyField)),
+                        mapEntryReflection,
+                        mapEntry,
+                        valueField);
+                    break;
+                  case FieldDescriptor::CPPTYPE_INT64:
+                    writeField(
+                        jsonify(
+                            mapEntryReflection->GetInt64(mapEntry, keyField)),
+                        mapEntryReflection,
+                        mapEntry,
+                        valueField);
+                    break;
+                  case FieldDescriptor::CPPTYPE_UINT32:
+                    writeField(
+                        jsonify(
+                            mapEntryReflection->GetUInt32(mapEntry, keyField)),
+                        mapEntryReflection,
+                        mapEntry,
+                        valueField);
+                    break;
+                  case FieldDescriptor::CPPTYPE_UINT64:
+                    writeField(
+                        jsonify(
+                            mapEntryReflection->GetUInt64(mapEntry, keyField)),
+                        mapEntryReflection,
+                        mapEntry,
+                        valueField);
+                    break;
+                  case FieldDescriptor::CPPTYPE_STRING:
+                    if (keyField->type() == FieldDescriptor::TYPE_BYTES) {
+                      LOG(FATAL)
+                        << "Unexpected key field type in protobuf Map: "
+                        << keyField->type_name();
+                    }
+
+                    writeField(
+                        mapEntryReflection->GetStringReference(
+                            mapEntry, keyField, nullptr),
+                        mapEntryReflection,
+                        mapEntry,
+                        valueField);
+                    break;
+                  case FieldDescriptor::CPPTYPE_FLOAT:
+                  case FieldDescriptor::CPPTYPE_DOUBLE:
+                  case FieldDescriptor::CPPTYPE_MESSAGE:
+                  case FieldDescriptor::CPPTYPE_ENUM:
+                    LOG(FATAL) << "Unexpected key field type in protobuf Map: "
+                               << keyField->cpp_type_name();
+                }
+              }
+            });
       }
     }
   }


[mesos] 02/03: Improved a test for jsonify map fields mapping.

Posted by mz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit fecbaf9f78b234c15b494303de0f2f37738ea583
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Mon Jul 29 14:26:48 2019 -0700

    Improved a test for jsonify map fields mapping.
    
    Review: https://reviews.apache.org/r/71186
---
 3rdparty/stout/tests/protobuf_tests.cpp | 126 +++++++++++++++++---------------
 1 file changed, 67 insertions(+), 59 deletions(-)

diff --git a/3rdparty/stout/tests/protobuf_tests.cpp b/3rdparty/stout/tests/protobuf_tests.cpp
index 95cdc67..55889dc 100644
--- a/3rdparty/stout/tests/protobuf_tests.cpp
+++ b/3rdparty/stout/tests/protobuf_tests.cpp
@@ -17,6 +17,8 @@
 #include <algorithm>
 #include <string>
 
+#include <google/protobuf/util/message_differencer.h>
+
 #include <stout/gtest.hpp>
 #include <stout/json.hpp>
 #include <stout/jsonify.hpp>
@@ -725,26 +727,27 @@ TEST(ProtobufTest, JsonifyLargeIntegers)
 TEST(ProtobufTest, JsonifyMap)
 {
   tests::MapMessage message;
+  (*message.mutable_string_to_string())["key1"] = "value1";
+  (*message.mutable_string_to_string())["key2"] = "value2";
   (*message.mutable_string_to_bool())["key1"] = true;
   (*message.mutable_string_to_bool())["key2"] = false;
   (*message.mutable_string_to_bytes())["key"] = "bytes";
-  (*message.mutable_string_to_double())["key"] = 1.0;
-  (*message.mutable_string_to_enum())["key"] = tests::ONE;
-  (*message.mutable_string_to_float())["key"] = 1.0;
-  (*message.mutable_string_to_string())["key1"] = "value1";
-  (*message.mutable_string_to_string())["key2"] = "value2";
-
-  tests::Nested nested;
-  nested.set_str("nested");
-  (*message.mutable_string_to_nested())["key"] = nested;
 
   // These numbers are equal or close to the integer limits.
   (*message.mutable_string_to_int32())["key"] = -2147483647;
   (*message.mutable_string_to_int64())["key"] = -9223372036854775807;
-  (*message.mutable_string_to_sint32())["key"] = -1234567890;
-  (*message.mutable_string_to_sint64())["key"] = -1234567890123456789;
   (*message.mutable_string_to_uint32())["key"] = 4294967295;
   (*message.mutable_string_to_uint64())["key"] = 9223372036854775807;
+  (*message.mutable_string_to_sint32())["key"] = -1234567890;
+  (*message.mutable_string_to_sint64())["key"] = -1234567890123456789;
+
+  (*message.mutable_string_to_float())["key"] = 1.0;
+  (*message.mutable_string_to_double())["key"] = 1.0;
+  (*message.mutable_string_to_enum())["key"] = tests::ONE;
+
+  tests::Nested nested;
+  nested.set_str("nested");
+  (*message.mutable_string_to_nested())["key"] = nested;
 
   (*message.mutable_bool_to_string())[true] = "value1";
   (*message.mutable_bool_to_string())[false] = "value2";
@@ -752,31 +755,19 @@ TEST(ProtobufTest, JsonifyMap)
   // These numbers are equal or close to the integer limits.
   (*message.mutable_int32_to_string())[-2147483647] = "value";
   (*message.mutable_int64_to_string())[-9223372036854775807] = "value";
-  (*message.mutable_sint32_to_string())[-1234567890] = "value";
-  (*message.mutable_sint64_to_string())[-1234567890123456789] = "value";
   (*message.mutable_uint32_to_string())[4294967295] = "value";
   (*message.mutable_uint64_to_string())[9223372036854775807] = "value";
+  (*message.mutable_sint32_to_string())[-1234567890] = "value";
+  (*message.mutable_sint64_to_string())[-1234567890123456789] = "value";
 
   // The keys are in alphabetical order.
   // The value of `string_to_bytes` is base64 encoded.
   string expected =
     R"~(
     {
-      "bool_to_string": {
-        "false": "value2",
-        "true": "value1"
-      },
-      "int32_to_string": {
-        "-2147483647": "value"
-      },
-      "int64_to_string": {
-        "-9223372036854775807": "value"
-      },
-      "sint32_to_string": {
-        "-1234567890": "value"
-      },
-      "sint64_to_string": {
-        "-1234567890123456789": "value"
+      "string_to_string": {
+        "key1": "value1",
+        "key2": "value2"
       },
       "string_to_bool": {
         "key1": true,
@@ -785,25 +776,17 @@ TEST(ProtobufTest, JsonifyMap)
       "string_to_bytes": {
         "key": "Ynl0ZXM="
       },
-      "string_to_double": {
-        "key": 1.0
-      },
-      "string_to_enum": {
-        "key": "ONE"
-      },
-      "string_to_float": {
-        "key": 1.0
-      },
       "string_to_int32": {
         "key": -2147483647
       },
       "string_to_int64": {
         "key": -9223372036854775807
       },
-      "string_to_nested": {
-        "key": {
-          "str": "nested"
-        }
+      "string_to_uint32": {
+        "key": 4294967295
+      },
+      "string_to_uint64": {
+        "key": 9223372036854775807
       },
       "string_to_sint32": {
         "key": -1234567890
@@ -811,36 +794,61 @@ TEST(ProtobufTest, JsonifyMap)
       "string_to_sint64": {
         "key": -1234567890123456789
       },
-      "string_to_string": {
-        "key1": "value1",
-        "key2": "value2"
+      "string_to_float": {
+        "key": 1.0
       },
-      "string_to_uint32": {
-        "key": 4294967295
+      "string_to_double": {
+        "key": 1.0
       },
-      "string_to_uint64": {
-        "key": 9223372036854775807
+      "string_to_enum": {
+        "key": "ONE"
+      },
+      "string_to_nested": {
+        "key": {
+          "str": "nested"
+        }
+      },
+      "bool_to_string": {
+        "false": "value2",
+        "true": "value1"
+      },
+      "int32_to_string": {
+        "-2147483647": "value"
+      },
+      "int64_to_string": {
+        "-9223372036854775807": "value"
       },
       "uint32_to_string": {
         "4294967295": "value"
       },
       "uint64_to_string": {
         "9223372036854775807": "value"
+      },
+      "sint32_to_string": {
+        "-1234567890": "value"
+      },
+      "sint64_to_string": {
+        "-1234567890123456789": "value"
       }
     })~";
 
-  // Remove ' ' and '\n' from `expected` so that we can compare
-  // it with the JSON string parsed from protobuf message.
-  expected.erase(
-      std::remove_if(expected.begin(), expected.end(), ::isspace),
-      expected.end());
+  // Entries within a map have no particular order, so we can't compare
+  // the string directly. Instead we parse the jsonify result back to
+  // a JSON value and compare with the parsed expected value.
+  Try<JSON::Value> jsonExpected = JSON::parse(expected);
+  ASSERT_SOME(jsonExpected);
 
-  JSON::Object object = JSON::protobuf(message);
-  EXPECT_EQ(expected, stringify(object));
+  Try<JSON::Value> jsonActual = JSON::parse(jsonify(JSON::Protobuf(message)));
+  ASSERT_SOME(jsonActual);
 
-  // Test parsing too.
-  Try<tests::MapMessage> parse = protobuf::parse<tests::MapMessage>(object);
-  ASSERT_SOME(parse);
+  EXPECT_EQ(*jsonExpected, *jsonActual);
 
-  EXPECT_EQ(object, JSON::protobuf(parse.get()));
+
+  // Test that we can map the json back to the expected protobuf.
+  Try<tests::MapMessage> protoFromJson =
+    protobuf::parse<tests::MapMessage>(*jsonActual);
+  ASSERT_SOME(protoFromJson);
+
+  EXPECT_TRUE(google::protobuf::util::MessageDifferencer::Equals(
+      message, *protoFromJson));
 }