internal/msgfmt: use msgfmt package to format messages

Port message formatting logic in testing/protocmp to internal/msgfmt
and improve upon its output.

This formatter is optimized for humanly readable output.
It takes the best parts of both the JSON and proto text formats.

The good of prototext:
	* It supports emitting unknown fields (useful for debugging).
	* It is relatively concise in the common-case since keys do not
	need to be represented as quoted strings (e.g., "key" vs key).

The bad of prototext:
	* Requires relatively large dependency on encoding/prototext.
	* Our implementation lacks support for serializing packed lists.
	* Lacks support for readable maps.
	* Lacks support for readable Timestamp and Duration.

The good of protojson:
	* First-class support for readable maps.
	* First-class support for readable Timestamp and Duration.

The bad of protojson:
	* Requires relatively large dependency on encoding/protojson.
	* Lacks support for emitting unknown fields.
	* More verbose in the common-case as keys are quoted strings.

The msgfmt package has the benefits of both protojson and prototext,
but none of the detriments. It is a relatively simple implementation.

This output is only intended for human consumption with no associated
deserialization implementation.
To avoid any illusion that this is identical to either the proto text
or JSON formats, this always emits surrounding "{}" for
top-level messages and the keys are not quoted strings.

This CL does not use this format for generated Message.String methods
as there is concerns about being inconsistent with the String methods
as implemented in other languages. Having it be a seperate package makes
it trivial to switch over to this if desired.

Change-Id: I8b3581904d1624e84bf1b1954d2f01e5774b7f87
Reviewed-on: https://go-review.googlesource.com/c/protobuf/+/223752
Reviewed-by: Damien Neil <dneil@google.com>
diff --git a/internal/msgfmt/format.go b/internal/msgfmt/format.go
new file mode 100644
index 0000000..79fe5ae
--- /dev/null
+++ b/internal/msgfmt/format.go
@@ -0,0 +1,283 @@
+// Copyright 2019 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package msgfmt implements a text marshaler combining the desirable features
+// of both the JSON and proto text formats.
+// It is optimized for human readability and has no associated deserializer.
+package msgfmt
+
+import (
+	"bytes"
+	"fmt"
+	"math"
+	"reflect"
+	"sort"
+	"strconv"
+	"strings"
+	"time"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/detrand"
+	"google.golang.org/protobuf/internal/mapsort"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+// Format returns a formatted string for the message.
+func Format(m proto.Message) string {
+	return string(appendMessage(nil, m.ProtoReflect()))
+}
+
+func appendValue(b []byte, v protoreflect.Value, fd protoreflect.FieldDescriptor) []byte {
+	switch v := v.Interface().(type) {
+	case bool, int32, int64, uint32, uint64, float32, float64:
+		return append(b, fmt.Sprint(v)...)
+	case string:
+		return append(b, strconv.Quote(string(v))...)
+	case []byte:
+		return append(b, strconv.Quote(string(v))...)
+	case protoreflect.EnumNumber:
+		return appendEnum(b, v, fd.Enum())
+	case protoreflect.Message:
+		return appendMessage(b, v)
+	case protoreflect.List:
+		return appendList(b, v, fd)
+	case protoreflect.Map:
+		return appendMap(b, v, fd)
+	default:
+		panic(fmt.Sprintf("invalid type: %T", v))
+	}
+}
+
+func appendEnum(b []byte, v protoreflect.EnumNumber, ed protoreflect.EnumDescriptor) []byte {
+	if ev := ed.Values().ByNumber(v); ev != nil {
+		return append(b, ev.Name()...)
+	}
+	return strconv.AppendInt(b, int64(v), 10)
+}
+
+func appendMessage(b []byte, m protoreflect.Message) []byte {
+	if b2 := appendKnownMessage(b, m); b2 != nil {
+		return b2
+	}
+
+	var fds []protoreflect.FieldDescriptor
+	m.Range(func(fd protoreflect.FieldDescriptor, _ protoreflect.Value) bool {
+		fds = append(fds, fd)
+		return true
+	})
+	sort.Slice(fds, func(i, j int) bool {
+		fdi, fdj := fds[i], fds[j]
+		switch {
+		case !fdi.IsExtension() && !fdj.IsExtension():
+			return fdi.Index() < fdj.Index()
+		case fdi.IsExtension() && fdj.IsExtension():
+			return fdi.FullName() < fdj.FullName()
+		default:
+			return !fdi.IsExtension() && fdj.IsExtension()
+		}
+	})
+
+	b = append(b, '{')
+	for _, fd := range fds {
+		k := string(fd.Name())
+		if fd.IsExtension() {
+			k = string("[" + fd.FullName() + "]")
+		}
+
+		b = append(b, k...)
+		b = append(b, ':')
+		b = appendValue(b, m.Get(fd), fd)
+		b = append(b, delim()...)
+	}
+	b = appendUnknown(b, m.GetUnknown())
+	b = bytes.TrimRight(b, delim())
+	b = append(b, '}')
+	return b
+}
+
+var protocmpMessageType = reflect.TypeOf(map[string]interface{}(nil))
+
+func appendKnownMessage(b []byte, m protoreflect.Message) []byte {
+	md := m.Descriptor()
+	if md.FullName().Parent() != "google.protobuf" {
+		return nil
+	}
+
+	fds := md.Fields()
+	switch md.Name() {
+	case "Any":
+		var msgVal protoreflect.Message
+		url := m.Get(fds.ByName("type_url")).String()
+		if v := reflect.ValueOf(m); v.Type().ConvertibleTo(protocmpMessageType) {
+			// For protocmp.Message, directly obtain the sub-message value
+			// which is stored in structured form, rather than as raw bytes.
+			m2 := v.Convert(protocmpMessageType).Interface().(map[string]interface{})
+			v, ok := m2["value"].(proto.Message)
+			if !ok {
+				return nil
+			}
+			msgVal = v.ProtoReflect()
+		} else {
+			val := m.Get(fds.ByName("value")).Bytes()
+			mt, err := protoregistry.GlobalTypes.FindMessageByURL(url)
+			if err != nil {
+				return nil
+			}
+			msgVal = mt.New()
+			err = proto.UnmarshalOptions{AllowPartial: true}.Unmarshal(val, msgVal.Interface())
+			if err != nil {
+				return nil
+			}
+		}
+
+		b = append(b, '{')
+		b = append(b, "["+url+"]"...)
+		b = append(b, ':')
+		b = appendMessage(b, msgVal)
+		b = append(b, '}')
+		return b
+
+	case "Timestamp":
+		const minTimestamp = -62135596800  // Seconds between 1970-01-01T00:00:00Z and 0001-01-01T00:00:00Z, inclusive
+		const maxTimestamp = +253402300799 // Seconds between 1970-01-01T00:00:00Z and 9999-12-31T23:59:59Z, inclusive
+		secs := m.Get(fds.ByName("seconds")).Int()
+		nanos := m.Get(fds.ByName("nanos")).Int()
+		switch {
+		case secs < minTimestamp || secs > maxTimestamp:
+			return nil
+		case nanos < 0 || nanos >= 1e9:
+			return nil
+		}
+
+		t := time.Unix(secs, nanos).UTC()
+		x := t.Format("2006-01-02T15:04:05.000000000") // RFC 3339
+		x = strings.TrimSuffix(x, "000")
+		x = strings.TrimSuffix(x, "000")
+		x = strings.TrimSuffix(x, ".000")
+		return append(b, x+"Z"...)
+
+	case "Duration":
+		const absDuration = 315576000000 // 10000yr * 365.25day/yr * 24hr/day * 60min/hr * 60sec/min
+		secs := m.Get(fds.ByName("seconds")).Int()
+		nanos := m.Get(fds.ByName("nanos")).Int()
+		switch {
+		case secs < -absDuration || secs > +absDuration:
+			return nil
+		case nanos <= -1e9 || nanos >= 1e9:
+			return nil
+		case (secs > 0 && nanos < 0) || (secs < 0 && nanos > 0):
+			return nil
+		}
+
+		x := fmt.Sprintf("%d.%09d", secs, int64(math.Abs(float64(nanos))))
+		x = strings.TrimSuffix(x, "000")
+		x = strings.TrimSuffix(x, "000")
+		x = strings.TrimSuffix(x, ".000")
+		return append(b, x+"s"...)
+
+	case "BoolValue", "Int32Value", "Int64Value", "UInt32Value", "UInt64Value", "FloatValue", "DoubleValue", "StringValue", "BytesValue":
+		fd := fds.ByName("value")
+		return appendValue(b, m.Get(fd), fd)
+	}
+
+	return nil
+}
+
+func appendUnknown(b []byte, raw protoreflect.RawFields) []byte {
+	rs := make(map[protoreflect.FieldNumber][]protoreflect.RawFields)
+	for len(raw) > 0 {
+		num, _, n := protowire.ConsumeField(raw)
+		rs[num] = append(rs[num], raw[:n])
+		raw = raw[n:]
+	}
+
+	var ns []protoreflect.FieldNumber
+	for n := range rs {
+		ns = append(ns, n)
+	}
+	sort.Slice(ns, func(i, j int) bool { return ns[i] < ns[j] })
+
+	for _, n := range ns {
+		var leftBracket, rightBracket string
+		if len(rs[n]) > 1 {
+			leftBracket, rightBracket = "[", "]"
+		}
+
+		b = strconv.AppendInt(b, int64(n), 10)
+		b = append(b, ':')
+		b = append(b, leftBracket...)
+		for _, r := range rs[n] {
+			num, typ, n := protowire.ConsumeTag(r)
+			r = r[n:]
+			switch typ {
+			case protowire.VarintType:
+				v, _ := protowire.ConsumeVarint(r)
+				b = strconv.AppendInt(b, int64(v), 10)
+			case protowire.Fixed32Type:
+				v, _ := protowire.ConsumeFixed32(r)
+				b = append(b, fmt.Sprintf("0x%08x", v)...)
+			case protowire.Fixed64Type:
+				v, _ := protowire.ConsumeFixed64(r)
+				b = append(b, fmt.Sprintf("0x%016x", v)...)
+			case protowire.BytesType:
+				v, _ := protowire.ConsumeBytes(r)
+				b = strconv.AppendQuote(b, string(v))
+			case protowire.StartGroupType:
+				v, _ := protowire.ConsumeGroup(num, r)
+				b = append(b, '{')
+				b = appendUnknown(b, v)
+				b = bytes.TrimRight(b, delim())
+				b = append(b, '}')
+			default:
+				panic(fmt.Sprintf("invalid type: %v", typ))
+			}
+			b = append(b, delim()...)
+		}
+		b = bytes.TrimRight(b, delim())
+		b = append(b, rightBracket...)
+		b = append(b, delim()...)
+	}
+	return b
+}
+
+func appendList(b []byte, v protoreflect.List, fd protoreflect.FieldDescriptor) []byte {
+	b = append(b, '[')
+	for i := 0; i < v.Len(); i++ {
+		b = appendValue(b, v.Get(i), fd)
+		b = append(b, delim()...)
+	}
+	b = bytes.TrimRight(b, delim())
+	b = append(b, ']')
+	return b
+}
+
+func appendMap(b []byte, v protoreflect.Map, fd protoreflect.FieldDescriptor) []byte {
+	var ks []protoreflect.MapKey
+	mapsort.Range(v, fd.MapKey().Kind(), func(k protoreflect.MapKey, _ protoreflect.Value) bool {
+		ks = append(ks, k)
+		return true
+	})
+
+	b = append(b, '{')
+	for _, k := range ks {
+		b = appendValue(b, k.Value(), fd.MapKey())
+		b = append(b, ':')
+		b = appendValue(b, v.Get(k), fd.MapValue())
+		b = append(b, delim()...)
+	}
+	b = bytes.TrimRight(b, delim())
+	b = append(b, '}')
+	return b
+}
+
+func delim() string {
+	// Deliberately introduce instability into the message string to
+	// discourage users from depending on it.
+	if detrand.Bool() {
+		return "  "
+	}
+	return ", "
+}
diff --git a/internal/msgfmt/format_test.go b/internal/msgfmt/format_test.go
new file mode 100644
index 0000000..6136d0e
--- /dev/null
+++ b/internal/msgfmt/format_test.go
@@ -0,0 +1,249 @@
+// Copyright 2020 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package msgfmt_test
+
+import (
+	"math"
+	"sync"
+	"testing"
+
+	"github.com/google/go-cmp/cmp"
+
+	"google.golang.org/protobuf/internal/detrand"
+	"google.golang.org/protobuf/internal/msgfmt"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/testing/protocmp"
+	"google.golang.org/protobuf/testing/protopack"
+
+	testpb "google.golang.org/protobuf/internal/testprotos/test"
+	textpb "google.golang.org/protobuf/internal/testprotos/textpb2"
+	dynpb "google.golang.org/protobuf/types/dynamicpb"
+	anypb "google.golang.org/protobuf/types/known/anypb"
+	durpb "google.golang.org/protobuf/types/known/durationpb"
+	tspb "google.golang.org/protobuf/types/known/timestamppb"
+	wpb "google.golang.org/protobuf/types/known/wrapperspb"
+)
+
+func init() {
+	detrand.Disable()
+}
+
+func TestFormat(t *testing.T) {
+	optMsg := &testpb.TestAllTypes{
+		OptionalBool:          proto.Bool(false),
+		OptionalInt32:         proto.Int32(-32),
+		OptionalInt64:         proto.Int64(-64),
+		OptionalUint32:        proto.Uint32(32),
+		OptionalUint64:        proto.Uint64(64),
+		OptionalFloat:         proto.Float32(32.32),
+		OptionalDouble:        proto.Float64(64.64),
+		OptionalString:        proto.String("string"),
+		OptionalBytes:         []byte("bytes"),
+		OptionalNestedEnum:    testpb.TestAllTypes_NEG.Enum(),
+		OptionalNestedMessage: &testpb.TestAllTypes_NestedMessage{A: proto.Int32(5)},
+	}
+	repMsg := &testpb.TestAllTypes{
+		RepeatedBool:   []bool{false, true},
+		RepeatedInt32:  []int32{32, -32},
+		RepeatedInt64:  []int64{64, -64},
+		RepeatedUint32: []uint32{0, 32},
+		RepeatedUint64: []uint64{0, 64},
+		RepeatedFloat:  []float32{0, 32.32},
+		RepeatedDouble: []float64{0, 64.64},
+		RepeatedString: []string{"s1", "s2"},
+		RepeatedBytes:  [][]byte{{1}, {2}},
+		RepeatedNestedEnum: []testpb.TestAllTypes_NestedEnum{
+			testpb.TestAllTypes_FOO,
+			testpb.TestAllTypes_BAR,
+		},
+		RepeatedNestedMessage: []*testpb.TestAllTypes_NestedMessage{
+			{A: proto.Int32(5)},
+			{A: proto.Int32(-5)},
+		},
+	}
+	mapMsg := &testpb.TestAllTypes{
+		MapBoolBool:     map[bool]bool{true: false},
+		MapInt32Int32:   map[int32]int32{-32: 32},
+		MapInt64Int64:   map[int64]int64{-64: 64},
+		MapUint32Uint32: map[uint32]uint32{0: 32},
+		MapUint64Uint64: map[uint64]uint64{0: 64},
+		MapInt32Float:   map[int32]float32{32: 32.32},
+		MapInt32Double:  map[int32]float64{64: 64.64},
+		MapStringString: map[string]string{"k": "v"},
+		MapStringBytes:  map[string][]byte{"k": []byte("v")},
+		MapStringNestedEnum: map[string]testpb.TestAllTypes_NestedEnum{
+			"k": testpb.TestAllTypes_FOO,
+		},
+		MapStringNestedMessage: map[string]*testpb.TestAllTypes_NestedMessage{
+			"k": {A: proto.Int32(5)},
+		},
+	}
+
+	tests := []struct {
+		in   proto.Message
+		want string
+	}{{
+		in:   optMsg,
+		want: `{optional_int32:-32, optional_int64:-64, optional_uint32:32, optional_uint64:64, optional_float:32.32, optional_double:64.64, optional_bool:false, optional_string:"string", optional_bytes:"bytes", optional_nested_message:{a:5}, optional_nested_enum:NEG}`,
+	}, {
+		in:   repMsg,
+		want: `{repeated_int32:[32, -32], repeated_int64:[64, -64], repeated_uint32:[0, 32], repeated_uint64:[0, 64], repeated_float:[0, 32.32], repeated_double:[0, 64.64], repeated_bool:[false, true], repeated_string:["s1", "s2"], repeated_bytes:["\x01", "\x02"], repeated_nested_message:[{a:5}, {a:-5}], repeated_nested_enum:[FOO, BAR]}`,
+	}, {
+		in:   mapMsg,
+		want: `{map_int32_int32:{-32:32}, map_int64_int64:{-64:64}, map_uint32_uint32:{0:32}, map_uint64_uint64:{0:64}, map_int32_float:{32:32.32}, map_int32_double:{64:64.64}, map_bool_bool:{true:false}, map_string_string:{"k":"v"}, map_string_bytes:{"k":"v"}, map_string_nested_message:{"k":{a:5}}, map_string_nested_enum:{"k":FOO}}`,
+	}, {
+		in: func() proto.Message {
+			m := &testpb.TestAllExtensions{}
+			proto.SetExtension(m, testpb.E_OptionalBool, bool(false))
+			proto.SetExtension(m, testpb.E_OptionalInt32, int32(-32))
+			proto.SetExtension(m, testpb.E_OptionalInt64, int64(-64))
+			proto.SetExtension(m, testpb.E_OptionalUint32, uint32(32))
+			proto.SetExtension(m, testpb.E_OptionalUint64, uint64(64))
+			proto.SetExtension(m, testpb.E_OptionalFloat, float32(32.32))
+			proto.SetExtension(m, testpb.E_OptionalDouble, float64(64.64))
+			proto.SetExtension(m, testpb.E_OptionalString, string("string"))
+			proto.SetExtension(m, testpb.E_OptionalBytes, []byte("bytes"))
+			proto.SetExtension(m, testpb.E_OptionalNestedEnum, testpb.TestAllTypes_NEG)
+			proto.SetExtension(m, testpb.E_OptionalNestedMessage, &testpb.TestAllExtensions_NestedMessage{A: proto.Int32(5)})
+			return m
+		}(),
+		want: `{[goproto.proto.test.optional_bool]:false, [goproto.proto.test.optional_bytes]:"bytes", [goproto.proto.test.optional_double]:64.64, [goproto.proto.test.optional_float]:32.32, [goproto.proto.test.optional_int32]:-32, [goproto.proto.test.optional_int64]:-64, [goproto.proto.test.optional_nested_enum]:NEG, [goproto.proto.test.optional_nested_message]:{a:5}, [goproto.proto.test.optional_string]:"string", [goproto.proto.test.optional_uint32]:32, [goproto.proto.test.optional_uint64]:64}`,
+	}, {
+		in: func() proto.Message {
+			m := &testpb.TestAllExtensions{}
+			proto.SetExtension(m, testpb.E_RepeatedBool, []bool{false, true})
+			proto.SetExtension(m, testpb.E_RepeatedInt32, []int32{32, -32})
+			proto.SetExtension(m, testpb.E_RepeatedInt64, []int64{64, -64})
+			proto.SetExtension(m, testpb.E_RepeatedUint32, []uint32{0, 32})
+			proto.SetExtension(m, testpb.E_RepeatedUint64, []uint64{0, 64})
+			proto.SetExtension(m, testpb.E_RepeatedFloat, []float32{0, 32.32})
+			proto.SetExtension(m, testpb.E_RepeatedDouble, []float64{0, 64.64})
+			proto.SetExtension(m, testpb.E_RepeatedString, []string{"s1", "s2"})
+			proto.SetExtension(m, testpb.E_RepeatedBytes, [][]byte{{1}, {2}})
+			proto.SetExtension(m, testpb.E_RepeatedNestedEnum, []testpb.TestAllTypes_NestedEnum{
+				testpb.TestAllTypes_FOO,
+				testpb.TestAllTypes_BAR,
+			})
+			proto.SetExtension(m, testpb.E_RepeatedNestedMessage, []*testpb.TestAllExtensions_NestedMessage{
+				{A: proto.Int32(5)},
+				{A: proto.Int32(-5)},
+			})
+			return m
+		}(),
+		want: `{[goproto.proto.test.repeated_bool]:[false, true], [goproto.proto.test.repeated_bytes]:["\x01", "\x02"], [goproto.proto.test.repeated_double]:[0, 64.64], [goproto.proto.test.repeated_float]:[0, 32.32], [goproto.proto.test.repeated_int32]:[32, -32], [goproto.proto.test.repeated_int64]:[64, -64], [goproto.proto.test.repeated_nested_enum]:[FOO, BAR], [goproto.proto.test.repeated_nested_message]:[{a:5}, {a:-5}], [goproto.proto.test.repeated_string]:["s1", "s2"], [goproto.proto.test.repeated_uint32]:[0, 32], [goproto.proto.test.repeated_uint64]:[0, 64]}`,
+	}, {
+		in: func() proto.Message {
+			m := &testpb.TestAllTypes{}
+			m.ProtoReflect().SetUnknown(protopack.Message{
+				protopack.Tag{Number: 50000, Type: protopack.VarintType}, protopack.Uvarint(100),
+				protopack.Tag{Number: 50001, Type: protopack.Fixed32Type}, protopack.Uint32(200),
+				protopack.Tag{Number: 50002, Type: protopack.Fixed64Type}, protopack.Uint64(300),
+				protopack.Tag{Number: 50003, Type: protopack.BytesType}, protopack.String("hello"),
+				protopack.Message{
+					protopack.Tag{Number: 50004, Type: protopack.StartGroupType},
+					protopack.Tag{Number: 1, Type: protopack.VarintType}, protopack.Uvarint(100),
+					protopack.Tag{Number: 1, Type: protopack.Fixed32Type}, protopack.Uint32(200),
+					protopack.Tag{Number: 1, Type: protopack.Fixed64Type}, protopack.Uint64(300),
+					protopack.Tag{Number: 1, Type: protopack.BytesType}, protopack.String("hello"),
+					protopack.Message{
+						protopack.Tag{Number: 1, Type: protopack.StartGroupType},
+						protopack.Tag{Number: 1, Type: protopack.VarintType}, protopack.Uvarint(100),
+						protopack.Tag{Number: 1, Type: protopack.Fixed32Type}, protopack.Uint32(200),
+						protopack.Tag{Number: 1, Type: protopack.Fixed64Type}, protopack.Uint64(300),
+						protopack.Tag{Number: 1, Type: protopack.BytesType}, protopack.String("hello"),
+						protopack.Tag{Number: 1, Type: protopack.EndGroupType},
+					},
+					protopack.Tag{Number: 50004, Type: protopack.EndGroupType},
+				},
+			}.Marshal())
+			return m
+		}(),
+		want: `{50000:100, 50001:0x000000c8, 50002:0x000000000000012c, 50003:"hello", 50004:{1:[100, 0x000000c8, 0x000000000000012c, "hello", {1:[100, 0x000000c8, 0x000000000000012c, "hello"]}]}}`,
+	}, {
+		in: &textpb.KnownTypes{
+			OptAny: &anypb.Any{
+				TypeUrl: "google.golang.org/goproto.proto.test.TestAllTypes",
+				Value: func() []byte {
+					b1, _ := proto.MarshalOptions{Deterministic: true}.Marshal(optMsg)
+					b2, _ := proto.MarshalOptions{Deterministic: true}.Marshal(repMsg)
+					b3, _ := proto.MarshalOptions{Deterministic: true}.Marshal(mapMsg)
+					return append(append(append([]byte(nil), b1...), b2...), b3...)
+				}(),
+			},
+		},
+		want: `{opt_any:{[google.golang.org/goproto.proto.test.TestAllTypes]:{optional_int32:-32, optional_int64:-64, optional_uint32:32, optional_uint64:64, optional_float:32.32, optional_double:64.64, optional_bool:false, optional_string:"string", optional_bytes:"bytes", optional_nested_message:{a:5}, optional_nested_enum:NEG, repeated_int32:[32, -32], repeated_int64:[64, -64], repeated_uint32:[0, 32], repeated_uint64:[0, 64], repeated_float:[0, 32.32], repeated_double:[0, 64.64], repeated_bool:[false, true], repeated_string:["s1", "s2"], repeated_bytes:["\x01", "\x02"], repeated_nested_message:[{a:5}, {a:-5}], repeated_nested_enum:[FOO, BAR], map_int32_int32:{-32:32}, map_int64_int64:{-64:64}, map_uint32_uint32:{0:32}, map_uint64_uint64:{0:64}, map_int32_float:{32:32.32}, map_int32_double:{64:64.64}, map_bool_bool:{true:false}, map_string_string:{"k":"v"}, map_string_bytes:{"k":"v"}, map_string_nested_message:{"k":{a:5}}, map_string_nested_enum:{"k":FOO}}}}`,
+	}, {
+		in: &textpb.KnownTypes{
+			OptTimestamp: &tspb.Timestamp{Seconds: math.MinInt64, Nanos: math.MaxInt32},
+		},
+		want: `{opt_timestamp:{seconds:-9223372036854775808, nanos:2147483647}}`,
+	}, {
+		in: &textpb.KnownTypes{
+			OptTimestamp: &tspb.Timestamp{Seconds: 1257894123, Nanos: 456789},
+		},
+		want: `{opt_timestamp:2009-11-10T23:02:03.000456789Z}`,
+	}, {
+		in: &textpb.KnownTypes{
+			OptDuration: &durpb.Duration{Seconds: math.MinInt64, Nanos: math.MaxInt32},
+		},
+		want: `{opt_duration:{seconds:-9223372036854775808, nanos:2147483647}}`,
+	}, {
+		in: &textpb.KnownTypes{
+			OptDuration: &durpb.Duration{Seconds: +1257894123, Nanos: +456789},
+		},
+		want: `{opt_duration:1257894123.000456789s}`,
+	}, {
+		in: &textpb.KnownTypes{
+			OptDuration: &durpb.Duration{Seconds: -1257894123, Nanos: -456789},
+		},
+		want: `{opt_duration:-1257894123.000456789s}`,
+	}, {
+		in: &textpb.KnownTypes{
+			OptBool:   &wpb.BoolValue{},
+			OptInt32:  &wpb.Int32Value{},
+			OptInt64:  &wpb.Int64Value{},
+			OptUint32: &wpb.UInt32Value{},
+			OptUint64: &wpb.UInt64Value{},
+			OptFloat:  &wpb.FloatValue{},
+			OptDouble: &wpb.DoubleValue{},
+			OptString: &wpb.StringValue{},
+			OptBytes:  &wpb.BytesValue{},
+		},
+		want: `{opt_bool:false, opt_int32:0, opt_int64:0, opt_uint32:0, opt_uint64:0, opt_float:0, opt_double:0, opt_string:"", opt_bytes:""}`,
+	}}
+	for _, tt := range tests {
+		t.Run("Generated", func(t *testing.T) {
+			got := msgfmt.Format(tt.in)
+			if diff := cmp.Diff(tt.want, got); diff != "" {
+				t.Errorf("Format() mismatch (-want +got):\n%v", diff)
+			}
+		})
+		t.Run("dynamicpb.Message", func(t *testing.T) {
+			m := dynpb.NewMessage(tt.in.ProtoReflect().Descriptor())
+			proto.Merge(m, tt.in)
+			got := msgfmt.Format(m)
+			if diff := cmp.Diff(tt.want, got); diff != "" {
+				t.Errorf("Format() mismatch (-want +got):\n%v", diff)
+			}
+		})
+		t.Run("protocmp.Message", func(t *testing.T) {
+			// This is a roundabout way to obtain a protocmp.Message since there
+			// is no exported API in protocmp to directly transform a message.
+			var m proto.Message
+			var once sync.Once
+			cmp.Equal(tt.in, tt.in, protocmp.Transform(), cmp.FilterPath(func(p cmp.Path) bool {
+				if v, _ := p.Index(1).Values(); v.IsValid() {
+					once.Do(func() { m = v.Interface().(protocmp.Message) })
+				}
+				return false
+			}, cmp.Ignore()))
+
+			got := msgfmt.Format(m)
+			if diff := cmp.Diff(tt.want, got); diff != "" {
+				t.Errorf("Format() mismatch (-want +got):\n%v", diff)
+			}
+		})
+	}
+}
diff --git a/testing/protocmp/xform.go b/testing/protocmp/xform.go
index 53dd8e6..78d43e3 100644
--- a/testing/protocmp/xform.go
+++ b/testing/protocmp/xform.go
@@ -17,6 +17,7 @@
 	"github.com/google/go-cmp/cmp"
 
 	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/msgfmt"
 	"google.golang.org/protobuf/proto"
 	"google.golang.org/protobuf/reflect/protoreflect"
 	"google.golang.org/protobuf/reflect/protoregistry"
@@ -139,7 +140,7 @@
 	case !m.ProtoReflect().IsValid():
 		return "<invalid>"
 	default:
-		return string(appendMessage(nil, m))
+		return msgfmt.Format(m)
 	}
 }
 
diff --git a/testing/protocmp/xform_test.go b/testing/protocmp/xform_test.go
index da6550b..7fc3f68 100644
--- a/testing/protocmp/xform_test.go
+++ b/testing/protocmp/xform_test.go
@@ -23,9 +23,8 @@
 
 func TestTransform(t *testing.T) {
 	tests := []struct {
-		in         proto.Message
-		want       Message
-		wantString string
+		in   proto.Message
+		want Message
 	}{{
 		in: &testpb.TestAllTypes{
 			OptionalBool:          proto.Bool(false),
@@ -54,7 +53,6 @@
 			"optional_nested_enum":    enumOf(testpb.TestAllTypes_NEG),
 			"optional_nested_message": Message{messageTypeKey: messageTypeOf(&testpb.TestAllTypes_NestedMessage{}), "a": int32(5)},
 		},
-		wantString: `{optional_int32:-32, optional_int64:-64, optional_uint32:32, optional_uint64:64, optional_float:32.32, optional_double:64.64, optional_bool:false, optional_string:"string", optional_bytes:"bytes", optional_nested_message:{a:5}, optional_nested_enum:NEG}`,
 	}, {
 		in: &testpb.TestAllTypes{
 			RepeatedBool:   []bool{false, true},
@@ -95,7 +93,6 @@
 				{messageTypeKey: messageTypeOf(&testpb.TestAllTypes_NestedMessage{}), "a": int32(-5)},
 			},
 		},
-		wantString: `{repeated_int32:[32, -32], repeated_int64:[64, -64], repeated_uint32:[0, 32], repeated_uint64:[0, 64], repeated_float:[0, 32.32], repeated_double:[0, 64.64], repeated_bool:[false, true], repeated_string:["s1", "s2"], repeated_bytes:["\x01", "\x02"], repeated_nested_message:[{a:5}, {a:-5}], repeated_nested_enum:[FOO, BAR]}`,
 	}, {
 		in: &testpb.TestAllTypes{
 			MapBoolBool:     map[bool]bool{true: false},
@@ -132,7 +129,6 @@
 				"k": {messageTypeKey: messageTypeOf(&testpb.TestAllTypes_NestedMessage{}), "a": int32(5)},
 			},
 		},
-		wantString: `{map_int32_int32:{-32:32}, map_int64_int64:{-64:64}, map_uint32_uint32:{0:32}, map_uint64_uint64:{0:64}, map_int32_float:{32:32.32}, map_int32_double:{64:64.64}, map_bool_bool:{true:false}, map_string_string:{"k":"v"}, map_string_bytes:{"k":"v"}, map_string_nested_message:{"k":{a:5}}, map_string_nested_enum:{"k":FOO}}`,
 	}, {
 		in: func() proto.Message {
 			m := &testpb.TestAllExtensions{}
@@ -163,7 +159,6 @@
 			"[goproto.proto.test.optional_nested_enum]":    enumOf(testpb.TestAllTypes_NEG),
 			"[goproto.proto.test.optional_nested_message]": Message{messageTypeKey: messageTypeOf(&testpb.TestAllExtensions_NestedMessage{}), "a": int32(5)},
 		},
-		wantString: `{[goproto.proto.test.optional_bool]:false, [goproto.proto.test.optional_bytes]:"bytes", [goproto.proto.test.optional_double]:64.64, [goproto.proto.test.optional_float]:32.32, [goproto.proto.test.optional_int32]:-32, [goproto.proto.test.optional_int64]:-64, [goproto.proto.test.optional_nested_enum]:NEG, [goproto.proto.test.optional_nested_message]:{a:5}, [goproto.proto.test.optional_string]:"string", [goproto.proto.test.optional_uint32]:32, [goproto.proto.test.optional_uint64]:64}`,
 	}, {
 		in: func() proto.Message {
 			m := &testpb.TestAllExtensions{}
@@ -206,7 +201,6 @@
 				{messageTypeKey: messageTypeOf(&testpb.TestAllExtensions_NestedMessage{}), "a": int32(-5)},
 			},
 		},
-		wantString: `{[goproto.proto.test.repeated_bool]:[false, true], [goproto.proto.test.repeated_bytes]:["\x01", "\x02"], [goproto.proto.test.repeated_double]:[0, 64.64], [goproto.proto.test.repeated_float]:[0, 32.32], [goproto.proto.test.repeated_int32]:[32, -32], [goproto.proto.test.repeated_int64]:[64, -64], [goproto.proto.test.repeated_nested_enum]:[FOO, BAR], [goproto.proto.test.repeated_nested_message]:[{a:5}, {a:-5}], [goproto.proto.test.repeated_string]:["s1", "s2"], [goproto.proto.test.repeated_uint32]:[0, 32], [goproto.proto.test.repeated_uint64]:[0, 64]}`,
 	}, {
 		in: func() proto.Message {
 			m := &testpb.TestAllTypes{}
@@ -257,7 +251,6 @@
 				protopack.Tag{Number: 50004, Type: protopack.EndGroupType},
 			}.Marshal()),
 		},
-		wantString: `{50000:100, 50001:200, 50002:300, 50003:"hello", 50004:{1:[100, 200, 300, "hello", {1:[100, 200, 300, "hello"]}]}}`,
 	}}
 	for _, tt := range tests {
 		t.Run("", func(t *testing.T) {
@@ -265,9 +258,6 @@
 			if diff := cmp.Diff(tt.want, got); diff != "" {
 				t.Errorf("Transform() mismatch (-want +got):\n%v", diff)
 			}
-			if diff := cmp.Diff(tt.wantString, got.String()); diff != "" {
-				t.Errorf("Transform().String() mismatch (-want +got):\n%v", diff)
-			}
 		})
 	}
 }