You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zipkin.apache.org by ad...@apache.org on 2019/05/11 11:23:57 UTC

[incubator-zipkin] 01/01: Folds Buffer tree into UnsafeBuffer to improve performance

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

adriancole pushed a commit to branch unsafe-buffer
in repository https://gitbox.apache.org/repos/asf/incubator-zipkin.git

commit 18e56355462fee54a77d9bcaf74dc01d1c5d7f2b
Author: Adrian Cole <ac...@pivotal.io>
AuthorDate: Sat May 11 19:18:25 2019 +0800

    Folds Buffer tree into UnsafeBuffer to improve performance
    
    Recent optimizations required peeking at the underlying byte array. This
    change formalizes the buffer as one that is specialized for being array
    backed (and writes are unsafe eg not bounds checked).
    
    The results are now almost the same as google's proto library, despite
    this library being smaller and including proto, json and thrift codecs
    and a functional Zipkin storage engine:
    
    Comparison
    ```
    ProtoCodecBenchmarks.bytes_protobufDecoder:bytes_protobufDecoder·p0.99       sample            1953.710                us/op
    ProtoCodecBenchmarks.bytes_protobufDecoder:·gc.alloc.rate.norm               sample     15  2023206.265 ±     32.203    B/op
    ProtoCodecBenchmarks.bytes_wireDecoder:bytes_wireDecoder·p0.99               sample            2351.104                us/op
    ProtoCodecBenchmarks.bytes_wireDecoder:·gc.alloc.rate.norm                   sample     15  2435408.568 ±     53.612    B/op
    ```
    
    Before
    ```
    ProtoCodecBenchmarks.bytes_zipkinDecoder:bytes_zipkinDecoder·p0.99           sample            2162.688                us/op
    ProtoCodecBenchmarks.bytes_zipkinDecoder:·gc.alloc.rate.norm                 sample     15  2095172.026 ±     45.379    B/op
    ```
    
    After
    ```
    ProtoCodecBenchmarks.bytes_zipkinDecoder:bytes_zipkinDecoder·p0.99           sample            1996.800                us/op
    ProtoCodecBenchmarks.bytes_zipkinDecoder:·gc.alloc.rate.norm                 sample     15  2095168.617 ±     36.985    B/op
    ```
    
    FYI: While this is array backed, it doesn't imply allocating a new array
    per span. Those looking to read large amounts of data can use this by
    slicing a buffer and filling for each length-prefixed byte string.
---
 ...Benchmarks.java => UnsafeBufferBenchmarks.java} |  16 +-
 .../zipkin2/internal/Proto3CodecInteropTest.java   |  22 +--
 .../zipkin2/server/internal/ZipkinQueryApiV2.java  |   8 +-
 .../zipkin2/codec/DependencyLinkBytesEncoder.java  |   8 +-
 .../java/zipkin2/internal/ByteArrayBuffer.java     | 102 -----------
 .../main/java/zipkin2/internal/Dependencies.java   |  12 +-
 .../src/main/java/zipkin2/internal/JsonCodec.java  |  22 +--
 .../main/java/zipkin2/internal/JsonEscaper.java    |   2 +-
 .../main/java/zipkin2/internal/Proto3Codec.java    |   6 +-
 .../main/java/zipkin2/internal/Proto3Fields.java   |  75 +++-----
 .../java/zipkin2/internal/Proto3SpanWriter.java    |  16 +-
 .../java/zipkin2/internal/Proto3ZipkinFields.java  |  24 +--
 .../main/java/zipkin2/internal/ThriftCodec.java    |  14 +-
 .../java/zipkin2/internal/ThriftEndpointCodec.java |   4 +-
 .../main/java/zipkin2/internal/ThriftField.java    |   2 +-
 .../internal/{Buffer.java => UnsafeBuffer.java}    | 193 +++++++++++++++------
 .../java/zipkin2/internal/V1JsonSpanWriter.java    |   4 +-
 .../main/java/zipkin2/internal/V1SpanWriter.java   |  12 +-
 .../java/zipkin2/internal/V1ThriftSpanWriter.java  |  28 +--
 .../main/java/zipkin2/internal/V2SpanWriter.java   |  10 +-
 .../test/java/zipkin2/internal/JsonCodecTest.java  |   9 +-
 .../java/zipkin2/internal/Proto3FieldsTest.java    |   4 +-
 .../zipkin2/internal/Proto3SpanWriterTest.java     |   6 +-
 .../zipkin2/internal/Proto3ZipkinFieldsTest.java   |  16 +-
 .../{BufferTest.java => UnsafeBufferTest.java}     |  92 +++++-----
 .../zipkin2/internal/V1JsonSpanWriterTest.java     |  22 +--
 .../zipkin2/internal/V1ThriftSpanWriterTest.java   |  48 ++---
 .../java/zipkin2/internal/V2SpanWriterTest.java    |  12 +-
 28 files changed, 379 insertions(+), 410 deletions(-)

diff --git a/benchmarks/src/main/java/zipkin2/internal/BufferBenchmarks.java b/benchmarks/src/main/java/zipkin2/internal/UnsafeBufferBenchmarks.java
similarity index 87%
rename from benchmarks/src/main/java/zipkin2/internal/BufferBenchmarks.java
rename to benchmarks/src/main/java/zipkin2/internal/UnsafeBufferBenchmarks.java
index e3a3a20..dac8060 100644
--- a/benchmarks/src/main/java/zipkin2/internal/BufferBenchmarks.java
+++ b/benchmarks/src/main/java/zipkin2/internal/UnsafeBufferBenchmarks.java
@@ -41,7 +41,7 @@ import org.openjdk.jmh.runner.options.OptionsBuilder;
 @OutputTimeUnit(TimeUnit.MICROSECONDS)
 @State(Scope.Thread)
 @Threads(1)
-public class BufferBenchmarks {
+public class UnsafeBufferBenchmarks {
   static final Charset UTF_8 = Charset.forName("UTF-8");
   // Order id = d07c4daa-0fa9-4c03-90b1-e06c4edae250 doesn't exist
   static final String CHINESE_UTF8 = "订单d07c4daa-0fa9-4c03-90b1-e06c4edae250不存在";
@@ -50,16 +50,16 @@ public class BufferBenchmarks {
   static final int TEST_INT = 1024;
   /* epoch micros timestamp */
   static final long TEST_LONG = 1472470996199000L;
-  Buffer buffer = Buffer.allocate(8);
+  UnsafeBuffer buffer = UnsafeBuffer.allocate(8);
 
   @Benchmark public int utf8SizeInBytes_chinese() {
-    return Buffer.utf8SizeInBytes(CHINESE_UTF8);
+    return UnsafeBuffer.utf8SizeInBytes(CHINESE_UTF8);
   }
 
   @Benchmark public byte[] writeUtf8_chinese() {
-    Buffer bufferUtf8 = Buffer.allocate(CHINESE_UTF8_SIZE);
+    UnsafeBuffer bufferUtf8 = UnsafeBuffer.allocate(CHINESE_UTF8_SIZE);
     bufferUtf8.writeUtf8(CHINESE_UTF8);
-    return bufferUtf8.toByteArrayUnsafe();
+    return bufferUtf8.unwrap();
   }
 
   @Benchmark public ByteBuffer writeUtf8_chinese_jdk() {
@@ -67,11 +67,11 @@ public class BufferBenchmarks {
   }
 
   @Benchmark public int varIntSizeInBytes_32() {
-    return Buffer.varintSizeInBytes(TEST_INT);
+    return UnsafeBuffer.varintSizeInBytes(TEST_INT);
   }
 
   @Benchmark public int varIntSizeInBytes_64() {
-    return Buffer.varintSizeInBytes(TEST_LONG);
+    return UnsafeBuffer.varintSizeInBytes(TEST_LONG);
   }
 
   @Benchmark public int writeVarint_32() {
@@ -95,7 +95,7 @@ public class BufferBenchmarks {
   // Convenience main entry-point
   public static void main(String[] args) throws RunnerException {
     Options opt = new OptionsBuilder()
-      .include(".*" + BufferBenchmarks.class.getSimpleName() + ".*")
+      .include(".*" + UnsafeBufferBenchmarks.class.getSimpleName() + ".*")
       .build();
 
     new Runner(opt).run();
diff --git a/benchmarks/src/test/java/zipkin2/internal/Proto3CodecInteropTest.java b/benchmarks/src/test/java/zipkin2/internal/Proto3CodecInteropTest.java
index 2a21379..d475538 100644
--- a/benchmarks/src/test/java/zipkin2/internal/Proto3CodecInteropTest.java
+++ b/benchmarks/src/test/java/zipkin2/internal/Proto3CodecInteropTest.java
@@ -156,10 +156,10 @@ public class Proto3CodecInteropTest {
     zipkin2.Annotation zipkinAnnotation = ZIPKIN_SPAN.annotations().get(0);
     Span wireSpan = new Span.Builder().annotations(PROTO_SPAN.annotations).build();
 
-    Buffer zipkinBytes = Buffer.allocate(ANNOTATION.sizeInBytes(zipkinAnnotation));
+    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(ANNOTATION.sizeInBytes(zipkinAnnotation));
     ANNOTATION.write(zipkinBytes, zipkinAnnotation);
 
-    assertThat(zipkinBytes.toByteArrayUnsafe())
+    assertThat(zipkinBytes.unwrap())
       .containsExactly(wireSpan.encode());
   }
 
@@ -167,7 +167,7 @@ public class Proto3CodecInteropTest {
     zipkin2.Annotation zipkinAnnotation = ZIPKIN_SPAN.annotations().get(0);
     Span wireSpan = new Span.Builder().annotations(PROTO_SPAN.annotations).build();
 
-    Buffer wireBytes = Buffer.wrap(wireSpan.encode(), 0);
+    UnsafeBuffer wireBytes = UnsafeBuffer.wrap(wireSpan.encode(), 0);
     assertThat(wireBytes.readVarint32())
       .isEqualTo(ANNOTATION.key);
 
@@ -188,20 +188,20 @@ public class Proto3CodecInteropTest {
   }
 
   @Test public void localEndpoint_write_matchesWire() {
-    Buffer zipkinBytes = Buffer.allocate(LOCAL_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.localEndpoint()));
+    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(LOCAL_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.localEndpoint()));
     LOCAL_ENDPOINT.write(zipkinBytes, ZIPKIN_SPAN.localEndpoint());
     Span wireSpan = new Span.Builder().local_endpoint(PROTO_SPAN.local_endpoint).build();
 
-    assertThat(zipkinBytes.toByteArrayUnsafe())
+    assertThat(zipkinBytes.unwrap())
       .containsExactly(wireSpan.encode());
   }
 
   @Test public void remoteEndpoint_write_matchesWire() {
-    Buffer zipkinBytes = Buffer.allocate(REMOTE_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.remoteEndpoint()));
+    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(REMOTE_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.remoteEndpoint()));
     REMOTE_ENDPOINT.write(zipkinBytes, ZIPKIN_SPAN.remoteEndpoint());
     Span wireSpan = new Span.Builder().remote_endpoint(PROTO_SPAN.remote_endpoint).build();
 
-    assertThat(zipkinBytes.toByteArrayUnsafe())
+    assertThat(zipkinBytes.unwrap())
       .containsExactly(wireSpan.encode());
   }
 
@@ -216,22 +216,22 @@ public class Proto3CodecInteropTest {
   @Test public void writeTagField_matchesWire() {
     MapEntry<String, String> entry = entry("clnt/finagle.version", "6.45.0");
     TagField field = new TagField(TAG_KEY);
-    Buffer zipkinBytes = Buffer.allocate(field.sizeInBytes(entry));
+    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(field.sizeInBytes(entry));
     field.write(zipkinBytes, entry);
 
     Span oneField = new Span.Builder().tags(singletonMap(entry.key, entry.value)).build();
-    assertThat(zipkinBytes.toByteArrayUnsafe())
+    assertThat(zipkinBytes.unwrap())
       .containsExactly(oneField.encode());
   }
 
   @Test public void writeTagField_matchesWire_emptyValue() {
     MapEntry<String, String> entry = entry("error", "");
     TagField field = new TagField(TAG_KEY);
-    Buffer zipkinBytes = Buffer.allocate(field.sizeInBytes(entry));
+    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(field.sizeInBytes(entry));
     field.write(zipkinBytes, entry);
 
     Span oneField = new Span.Builder().tags(singletonMap(entry.key, entry.value)).build();
-    assertThat(zipkinBytes.toByteArrayUnsafe())
+    assertThat(zipkinBytes.unwrap())
       .containsExactly(oneField.encode());
   }
 
diff --git a/zipkin-server/src/main/java/zipkin2/server/internal/ZipkinQueryApiV2.java b/zipkin-server/src/main/java/zipkin2/server/internal/ZipkinQueryApiV2.java
index 8cef06c..265a89b 100644
--- a/zipkin-server/src/main/java/zipkin2/server/internal/ZipkinQueryApiV2.java
+++ b/zipkin-server/src/main/java/zipkin2/server/internal/ZipkinQueryApiV2.java
@@ -39,8 +39,8 @@ import zipkin2.DependencyLink;
 import zipkin2.Span;
 import zipkin2.codec.DependencyLinkBytesEncoder;
 import zipkin2.codec.SpanBytesEncoder;
-import zipkin2.internal.Buffer;
 import zipkin2.internal.JsonCodec;
+import zipkin2.internal.UnsafeBuffer;
 import zipkin2.storage.QueryRequest;
 import zipkin2.storage.StorageComponent;
 
@@ -147,12 +147,12 @@ public class ZipkinQueryApiV2 {
       .setInt(HttpHeaderNames.CONTENT_LENGTH, body.length).build(), HttpData.of(body));
   }
 
-  static final Buffer.Writer<String> QUOTED_STRING_WRITER = new Buffer.Writer<String>() {
+  static final UnsafeBuffer.Writer<String> QUOTED_STRING_WRITER = new UnsafeBuffer.Writer<String>() {
     @Override public int sizeInBytes(String value) {
-      return Buffer.utf8SizeInBytes(value) + 2; // quotes
+      return UnsafeBuffer.utf8SizeInBytes(value) + 2; // quotes
     }
 
-    @Override public void write(String value, Buffer buffer) {
+    @Override public void write(String value, UnsafeBuffer buffer) {
       buffer.writeByte('"');
       buffer.writeUtf8(value);
       buffer.writeByte('"');
diff --git a/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java b/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java
index b28b139..aee13b5 100644
--- a/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java
+++ b/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java
@@ -18,12 +18,12 @@ package zipkin2.codec;
 
 import java.util.List;
 import zipkin2.DependencyLink;
-import zipkin2.internal.Buffer;
 import zipkin2.internal.JsonCodec;
+import zipkin2.internal.UnsafeBuffer;
 
-import static zipkin2.internal.Buffer.asciiSizeInBytes;
 import static zipkin2.internal.JsonEscaper.jsonEscape;
 import static zipkin2.internal.JsonEscaper.jsonEscapedSizeInBytes;
+import static zipkin2.internal.UnsafeBuffer.asciiSizeInBytes;
 
 public enum DependencyLinkBytesEncoder implements BytesEncoder<DependencyLink> {
   JSON_V1 {
@@ -44,7 +44,7 @@ public enum DependencyLinkBytesEncoder implements BytesEncoder<DependencyLink> {
     }
   };
 
-  static final Buffer.Writer<DependencyLink> WRITER = new Buffer.Writer<DependencyLink>() {
+  static final UnsafeBuffer.Writer<DependencyLink> WRITER = new UnsafeBuffer.Writer<DependencyLink>() {
     @Override public int sizeInBytes(DependencyLink value) {
       int sizeInBytes = 37; // {"parent":"","child":"","callCount":}
       sizeInBytes += jsonEscapedSizeInBytes(value.parent());
@@ -57,7 +57,7 @@ public enum DependencyLinkBytesEncoder implements BytesEncoder<DependencyLink> {
       return sizeInBytes;
     }
 
-    @Override public void write(DependencyLink value, Buffer b) {
+    @Override public void write(DependencyLink value, UnsafeBuffer b) {
       b.writeAscii("{\"parent\":\"");
       b.writeUtf8(jsonEscape(value.parent()));
       b.writeAscii("\",\"child\":\"");
diff --git a/zipkin/src/main/java/zipkin2/internal/ByteArrayBuffer.java b/zipkin/src/main/java/zipkin2/internal/ByteArrayBuffer.java
deleted file mode 100644
index 51f8cda..0000000
--- a/zipkin/src/main/java/zipkin2/internal/ByteArrayBuffer.java
+++ /dev/null
@@ -1,102 +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 zipkin2.internal;
-
-final class ByteArrayBuffer extends Buffer {
-
-  private final byte[] buf;
-  int pos; // visible for testing
-
-  ByteArrayBuffer(int size) {
-    buf = new byte[size];
-  }
-
-  ByteArrayBuffer(byte[] buf, int pos) {
-    this.buf = buf;
-    this.pos = pos;
-  }
-
-  @Override public void writeByte(int v) {
-    buf[pos++] = (byte) (v & 0xff);
-  }
-
-  @Override public void write(byte[] v) {
-    System.arraycopy(v, 0, buf, pos, v.length);
-    pos += v.length;
-  }
-
-  @Override void writeBackwards(long v) {
-    int pos = this.pos += asciiSizeInBytes(v); // We write backwards from right to left.
-    while (v != 0) {
-      int digit = (int) (v % 10);
-      buf[--pos] = DIGITS[digit];
-      v /= 10;
-    }
-  }
-
-  /** Inspired by {@code okio.Buffer.writeLong} */
-  @Override public void writeLongHex(long v) {
-    writeHexByte(buf, pos + 0, (byte) ((v >>> 56L) & 0xff));
-    writeHexByte(buf, pos + 2, (byte) ((v >>> 48L) & 0xff));
-    writeHexByte(buf, pos + 4, (byte) ((v >>> 40L) & 0xff));
-    writeHexByte(buf, pos + 6, (byte) ((v >>> 32L) & 0xff));
-    writeHexByte(buf, pos + 8, (byte) ((v >>> 24L) & 0xff));
-    writeHexByte(buf, pos + 10, (byte) ((v >>> 16L) & 0xff));
-    writeHexByte(buf, pos + 12, (byte) ((v >>> 8L) & 0xff));
-    writeHexByte(buf, pos + 14, (byte) (v & 0xff));
-    pos += 16;
-  }
-
-  @Override public void reset() {
-    pos = 0;
-  }
-
-  @Override byte readByteUnsafe() {
-    return buf[pos++];
-  }
-
-  @Override byte[] readByteArray(int length) {
-    ensureLength(this, length);
-    byte[] result = new byte[length];
-    System.arraycopy(buf, pos, result, 0, length);
-    pos += length;
-    return result;
-  }
-
-  @Override int remaining() {
-    return buf.length - pos;
-  }
-
-  @Override boolean skip(int maxCount) {
-    int nextPos = pos + maxCount;
-    if (nextPos > buf.length) {
-      pos = buf.length;
-      return false;
-    }
-    pos = nextPos;
-    return true;
-  }
-
-  @Override public int pos() {
-    return pos;
-  }
-
-  @Override public byte[] toByteArrayUnsafe() {
-    // assert pos == buf.length;
-    return buf;
-  }
-}
diff --git a/zipkin/src/main/java/zipkin2/internal/Dependencies.java b/zipkin/src/main/java/zipkin2/internal/Dependencies.java
index 8cd1709..6141ed5 100644
--- a/zipkin/src/main/java/zipkin2/internal/Dependencies.java
+++ b/zipkin/src/main/java/zipkin2/internal/Dependencies.java
@@ -22,12 +22,12 @@ import java.util.Collections;
 import java.util.List;
 import zipkin2.DependencyLink;
 
-import static zipkin2.internal.Buffer.utf8SizeInBytes;
 import static zipkin2.internal.ThriftCodec.skip;
 import static zipkin2.internal.ThriftField.TYPE_I64;
 import static zipkin2.internal.ThriftField.TYPE_LIST;
 import static zipkin2.internal.ThriftField.TYPE_STOP;
 import static zipkin2.internal.ThriftField.TYPE_STRING;
+import static zipkin2.internal.UnsafeBuffer.utf8SizeInBytes;
 
 /**
  * Internal as only cassandra serializes the start and end timestamps along with link data, and
@@ -76,9 +76,9 @@ public final class Dependencies {
 
   /** Writes the current instance in TBinaryProtocol */
   public ByteBuffer toThrift() {
-    Buffer buffer = Buffer.allocate(sizeInBytes());
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(sizeInBytes());
     write(buffer);
-    return ByteBuffer.wrap(buffer.toByteArrayUnsafe());
+    return ByteBuffer.wrap(buffer.unwrap());
   }
 
   int sizeInBytes() {
@@ -90,7 +90,7 @@ public final class Dependencies {
     return sizeInBytes;
   }
 
-  void write(Buffer buffer) {
+  void write(UnsafeBuffer buffer) {
     START_TS.write(buffer);
     ThriftCodec.writeLong(buffer, startTs);
 
@@ -138,7 +138,7 @@ public final class Dependencies {
     return h;
   }
 
-  static final class DependencyLinkAdapter implements Buffer.Writer<DependencyLink> {
+  static final class DependencyLinkAdapter implements UnsafeBuffer.Writer<DependencyLink> {
 
     static final ThriftField PARENT = new ThriftField(TYPE_STRING, 1);
     static final ThriftField CHILD = new ThriftField(TYPE_STRING, 2);
@@ -181,7 +181,7 @@ public final class Dependencies {
     }
 
     @Override
-    public void write(DependencyLink value, Buffer buffer) {
+    public void write(DependencyLink value, UnsafeBuffer buffer) {
       PARENT.write(buffer);
       ThriftCodec.writeLengthPrefixed(buffer, value.parent());
 
diff --git a/zipkin/src/main/java/zipkin2/internal/JsonCodec.java b/zipkin/src/main/java/zipkin2/internal/JsonCodec.java
index cce9f9a..0ecf4cc 100644
--- a/zipkin/src/main/java/zipkin2/internal/JsonCodec.java
+++ b/zipkin/src/main/java/zipkin2/internal/JsonCodec.java
@@ -157,7 +157,7 @@ public final class JsonCodec {
     }
   }
 
-  static <T> int sizeInBytes(Buffer.Writer<T> writer, List<T> value) {
+  static <T> int sizeInBytes(UnsafeBuffer.Writer<T> writer, List<T> value) {
     int length = value.size();
     int sizeInBytes = 2; // []
     if (length > 1) sizeInBytes += length - 1; // comma to join elements
@@ -168,12 +168,12 @@ public final class JsonCodec {
   }
 
   /** Inability to encode is a programming bug. */
-  public static <T> byte[] write(Buffer.Writer<T> writer, T value) {
-    Buffer b = Buffer.allocate(writer.sizeInBytes(value));
+  public static <T> byte[] write(UnsafeBuffer.Writer<T> writer, T value) {
+    UnsafeBuffer b = UnsafeBuffer.allocate(writer.sizeInBytes(value));
     try {
       writer.write(value, b);
     } catch (RuntimeException e) {
-      byte[] bytes = b.toByteArrayUnsafe();
+      byte[] bytes = b.unwrap();
       int lengthWritten = bytes.length;
       for (int i = 0; i < bytes.length; i++) {
         if (bytes[i] == 0) {
@@ -194,29 +194,29 @@ public final class JsonCodec {
           new String(bytes, 0, lengthWritten, UTF_8));
       throw Platform.get().assertionError(message, e);
     }
-    return b.toByteArrayUnsafe();
+    return b.unwrap();
   }
 
-  public static <T> byte[] writeList(Buffer.Writer<T> writer, List<T> value) {
+  public static <T> byte[] writeList(UnsafeBuffer.Writer<T> writer, List<T> value) {
     if (value.isEmpty()) return new byte[] {'[', ']'};
-    Buffer result = Buffer.allocate(sizeInBytes(writer, value));
+    UnsafeBuffer result = UnsafeBuffer.allocate(sizeInBytes(writer, value));
     writeList(writer, value, result);
-    return result.toByteArrayUnsafe();
+    return result.unwrap();
   }
 
-  public static <T> int writeList(Buffer.Writer<T> writer, List<T> value, byte[] out, int pos) {
+  public static <T> int writeList(UnsafeBuffer.Writer<T> writer, List<T> value, byte[] out, int pos) {
     if (value.isEmpty()) {
       out[pos++] = '[';
       out[pos++] = ']';
       return 2;
     }
     int initialPos = pos;
-    Buffer result = Buffer.wrap(out, pos);
+    UnsafeBuffer result = UnsafeBuffer.wrap(out, pos);
     writeList(writer, value, result);
     return result.pos() - initialPos;
   }
 
-  public static <T> void writeList(Buffer.Writer<T> writer, List<T> value, Buffer b) {
+  public static <T> void writeList(UnsafeBuffer.Writer<T> writer, List<T> value, UnsafeBuffer b) {
     b.writeByte('[');
     for (int i = 0, length = value.size(); i < length; ) {
       writer.write(value.get(i++), b);
diff --git a/zipkin/src/main/java/zipkin2/internal/JsonEscaper.java b/zipkin/src/main/java/zipkin2/internal/JsonEscaper.java
index bfd143e..77f0093 100644
--- a/zipkin/src/main/java/zipkin2/internal/JsonEscaper.java
+++ b/zipkin/src/main/java/zipkin2/internal/JsonEscaper.java
@@ -99,6 +99,6 @@ public final class JsonEscaper {
       }
     }
     if (ascii) return v.length() + escapingOverhead;
-    return Buffer.utf8SizeInBytes(v) + escapingOverhead;
+    return UnsafeBuffer.utf8SizeInBytes(v) + escapingOverhead;
   }
 }
diff --git a/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java b/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java
index 1fcd640..c5ae215 100644
--- a/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java
+++ b/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java
@@ -46,7 +46,7 @@ public final class Proto3Codec {
 
   public static boolean read(byte[] bytes, Collection<Span> out) {
     if (bytes.length == 0) return false;
-    Buffer buffer = Buffer.wrap(bytes, 0);
+    UnsafeBuffer buffer = UnsafeBuffer.wrap(bytes, 0);
     try {
       Span span = SPAN.read(buffer);
       if (span == null) return false;
@@ -58,13 +58,13 @@ public final class Proto3Codec {
   }
 
   public static @Nullable Span readOne(byte[] bytes) {
-    return SPAN.read(Buffer.wrap(bytes, 0));
+    return SPAN.read(UnsafeBuffer.wrap(bytes, 0));
   }
 
   public static boolean readList(byte[] bytes, Collection<Span> out) {
     int length = bytes.length;
     if (length == 0) return false;
-    Buffer buffer = Buffer.wrap(bytes, 0);
+    UnsafeBuffer buffer = UnsafeBuffer.wrap(bytes, 0);
     try {
       while (buffer.pos() < length) {
         Span span = SPAN.read(buffer);
diff --git a/zipkin/src/main/java/zipkin2/internal/Proto3Fields.java b/zipkin/src/main/java/zipkin2/internal/Proto3Fields.java
index ba2df27..0761795 100644
--- a/zipkin/src/main/java/zipkin2/internal/Proto3Fields.java
+++ b/zipkin/src/main/java/zipkin2/internal/Proto3Fields.java
@@ -18,8 +18,6 @@ package zipkin2.internal;
 
 import zipkin2.Endpoint;
 
-import static zipkin2.internal.JsonCodec.UTF_8;
-
 /**
  * Everything here assumes the field numbers are less than 16, implying a 1 byte tag.
  */
@@ -40,7 +38,8 @@ final class Proto3Fields {
     final int fieldNumber;
     final int wireType;
     /**
-     * "Each key in the streamed message is a varint with the value {@code (field_number << 3) | wire_type}"
+     * "Each key in the streamed message is a varint with the value {@code (field_number << 3) |
+     * wire_type}"
      *
      * <p>See https://developers.google.com/protocol-buffers/docs/encoding#structure
      */
@@ -71,7 +70,7 @@ final class Proto3Fields {
       return wireType;
     }
 
-    static boolean skipValue(Buffer buffer, int wireType) {
+    static boolean skipValue(UnsafeBuffer buffer, int wireType) {
       int remaining = buffer.remaining();
       switch (wireType) {
         case WIRETYPE_VARINT:
@@ -112,7 +111,7 @@ final class Proto3Fields {
       return sizeOfLengthDelimitedField(sizeOfValue);
     }
 
-    final void write(Buffer b, T value) {
+    final void write(UnsafeBuffer b, T value) {
       if (value == null) return;
       int sizeOfValue = sizeOfValue(value);
       b.writeByte(key);
@@ -124,22 +123,22 @@ final class Proto3Fields {
      * Calling this after consuming the field key to ensures there's enough space for the data. Null
      * is returned when the length prefix is zero.
      */
-    final T readLengthPrefixAndValue(Buffer b) {
+    final T readLengthPrefixAndValue(UnsafeBuffer b) {
       int length = readLengthPrefix(b);
       if (length == 0) return null;
       return readValue(b, length);
     }
 
-    final int readLengthPrefix(Buffer b) {
+    final int readLengthPrefix(UnsafeBuffer b) {
       return b.readVarint32();
     }
 
     abstract int sizeOfValue(T value);
 
-    abstract void writeValue(Buffer b, T value);
+    abstract void writeValue(UnsafeBuffer b, T value);
 
     /** @param length is greater than zero */
-    abstract T readValue(Buffer b, int length);
+    abstract T readValue(UnsafeBuffer b, int length);
   }
 
   static class BytesField extends LengthDelimitedField<byte[]> {
@@ -151,19 +150,16 @@ final class Proto3Fields {
       return bytes.length;
     }
 
-    @Override void writeValue(Buffer b, byte[] bytes) {
+    @Override void writeValue(UnsafeBuffer b, byte[] bytes) {
       b.write(bytes);
     }
 
-    @Override byte[] readValue(Buffer b, int length) {
-      return b.readByteArray(length);
+    @Override byte[] readValue(UnsafeBuffer b, int length) {
+      return b.readBytes(length);
     }
   }
 
   static class HexField extends LengthDelimitedField<String> {
-    static final char[] HEX_DIGITS =
-      {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'};
-
     HexField(int key) {
       super(key);
     }
@@ -173,7 +169,7 @@ final class Proto3Fields {
       return hex.length() / 2;
     }
 
-    @Override void writeValue(Buffer b, String hex) {
+    @Override void writeValue(UnsafeBuffer b, String hex) {
       // similar logic to okio.ByteString.decodeHex
       for (int i = 0, length = hex.length(); i < length; i++) {
         int d1 = decodeLowerHex(hex.charAt(i++)) << 4;
@@ -188,23 +184,8 @@ final class Proto3Fields {
       throw new AssertionError("not lowerHex " + c); // bug
     }
 
-    @Override String readValue(Buffer buffer, int length) {
-      length *= 2;
-
-      // All our hex fields are at most 32 characters.
-      if (length > 32) {
-        throw new IllegalArgumentException("hex field greater than 32 chars long: " + length);
-      }
-
-      char[] result = Platform.get().idBuffer();
-
-      for (int i = 0; i < length; i += 2) {
-        byte b = buffer.readByte();
-        result[i + 0] = HEX_DIGITS[(b >> 4) & 0xf];
-        result[i + 1] = HEX_DIGITS[b & 0xf];
-      }
-
-      return new String(result, 0, length);
+    @Override String readValue(UnsafeBuffer buffer, int length) {
+      return buffer.readBytesAsHex(length);
     }
   }
 
@@ -214,17 +195,15 @@ final class Proto3Fields {
     }
 
     @Override int sizeOfValue(String utf8) {
-      return utf8 != null ? Buffer.utf8SizeInBytes(utf8) : 0;
+      return utf8 != null ? UnsafeBuffer.utf8SizeInBytes(utf8) : 0;
     }
 
-    @Override void writeValue(Buffer b, String utf8) {
+    @Override void writeValue(UnsafeBuffer b, String utf8) {
       b.writeUtf8(utf8);
     }
 
-    @Override String readValue(Buffer buffer, int length) {
-      String result = new String(buffer.toByteArrayUnsafe(), buffer.pos(), length, UTF_8);
-      buffer.skip(length);
-      return result;
+    @Override String readValue(UnsafeBuffer buffer, int length) {
+      return buffer.readUtf8(length);
     }
   }
 
@@ -234,7 +213,7 @@ final class Proto3Fields {
       assert wireType == WIRETYPE_FIXED64;
     }
 
-    void write(Buffer b, long number) {
+    void write(UnsafeBuffer b, long number) {
       if (number == 0) return;
       b.writeByte(key);
       b.writeLongLe(number);
@@ -245,7 +224,7 @@ final class Proto3Fields {
       return 1 + 8; // tag + 8 byte number
     }
 
-    long readValue(Buffer buffer) {
+    long readValue(UnsafeBuffer buffer) {
       return buffer.readLongLe();
     }
   }
@@ -257,20 +236,20 @@ final class Proto3Fields {
     }
 
     int sizeInBytes(int number) {
-      return number != 0 ? 1 + Buffer.varintSizeInBytes(number) : 0; // tag + varint
+      return number != 0 ? 1 + UnsafeBuffer.varintSizeInBytes(number) : 0; // tag + varint
     }
 
-    void write(Buffer b, int number) {
+    void write(UnsafeBuffer b, int number) {
       if (number == 0) return;
       b.writeByte(key);
       b.writeVarint(number);
     }
 
     int sizeInBytes(long number) {
-      return number != 0 ? 1 + Buffer.varintSizeInBytes(number) : 0; // tag + varint
+      return number != 0 ? 1 + UnsafeBuffer.varintSizeInBytes(number) : 0; // tag + varint
     }
 
-    void write(Buffer b, long number) {
+    void write(UnsafeBuffer b, long number) {
       if (number == 0) return;
       b.writeByte(key);
       b.writeVarint(number);
@@ -287,13 +266,13 @@ final class Proto3Fields {
       return bool ? 2 : 0; // tag + varint
     }
 
-    void write(Buffer b, boolean bool) {
+    void write(UnsafeBuffer b, boolean bool) {
       if (!bool) return;
       b.writeByte(key);
       b.writeByte(1);
     }
 
-    boolean read(Buffer b) {
+    boolean read(UnsafeBuffer b) {
       byte bool = b.readByte();
       if (bool < 0 || bool > 1) {
         throw new IllegalArgumentException("Malformed: invalid boolean value at byte " + b.pos());
@@ -316,6 +295,6 @@ final class Proto3Fields {
   }
 
   static int sizeOfLengthDelimitedField(int sizeInBytes) {
-    return 1 + Buffer.varintSizeInBytes(sizeInBytes) + sizeInBytes; // tag + len + bytes
+    return 1 + UnsafeBuffer.varintSizeInBytes(sizeInBytes) + sizeInBytes; // tag + len + bytes
   }
 }
diff --git a/zipkin/src/main/java/zipkin2/internal/Proto3SpanWriter.java b/zipkin/src/main/java/zipkin2/internal/Proto3SpanWriter.java
index cc6cf4d..8e2d8cd 100644
--- a/zipkin/src/main/java/zipkin2/internal/Proto3SpanWriter.java
+++ b/zipkin/src/main/java/zipkin2/internal/Proto3SpanWriter.java
@@ -23,7 +23,7 @@ import static zipkin2.internal.Proto3Fields.sizeOfLengthDelimitedField;
 import static zipkin2.internal.Proto3ZipkinFields.SPAN;
 
 //@Immutable
-final class Proto3SpanWriter implements Buffer.Writer<Span> {
+final class Proto3SpanWriter implements UnsafeBuffer.Writer<Span> {
 
   static final byte[] EMPTY_ARRAY = new byte[0];
 
@@ -31,7 +31,7 @@ final class Proto3SpanWriter implements Buffer.Writer<Span> {
     return SPAN.sizeInBytes(span);
   }
 
-  @Override public void write(Span value, Buffer b) {
+  @Override public void write(Span value, UnsafeBuffer b) {
     SPAN.write(b, value);
   }
 
@@ -51,22 +51,22 @@ final class Proto3SpanWriter implements Buffer.Writer<Span> {
       int sizeOfValue = sizeOfValues[i] = SPAN.sizeOfValue(spans.get(i));
       sizeInBytes += sizeOfLengthDelimitedField(sizeOfValue);
     }
-    Buffer result = Buffer.allocate(sizeInBytes);
+    UnsafeBuffer result = UnsafeBuffer.allocate(sizeInBytes);
     for (int i = 0; i < lengthOfSpans; i++) {
       writeSpan(spans.get(i), sizeOfValues[i], result);
     }
-    return result.toByteArrayUnsafe();
+    return result.unwrap();
   }
 
   byte[] write(Span onlySpan) {
     int sizeOfValue = SPAN.sizeOfValue(onlySpan);
-    Buffer result = Buffer.allocate(sizeOfLengthDelimitedField(sizeOfValue));
+    UnsafeBuffer result = UnsafeBuffer.allocate(sizeOfLengthDelimitedField(sizeOfValue));
     writeSpan(onlySpan, sizeOfValue, result);
-    return result.toByteArrayUnsafe();
+    return result.unwrap();
   }
 
   // prevents resizing twice
-  void writeSpan(Span span, int sizeOfSpan, Buffer result) {
+  void writeSpan(Span span, int sizeOfSpan, UnsafeBuffer result) {
     result.writeByte(SPAN.key);
     result.writeVarint(sizeOfSpan); // length prefix
     SPAN.writeValue(result, span);
@@ -76,7 +76,7 @@ final class Proto3SpanWriter implements Buffer.Writer<Span> {
     int lengthOfSpans = spans.size();
     if (lengthOfSpans == 0) return 0;
 
-    Buffer result = Buffer.wrap(out, pos);
+    UnsafeBuffer result = UnsafeBuffer.wrap(out, pos);
     for (int i = 0; i < lengthOfSpans; i++) {
       SPAN.write(result, spans.get(i));
     }
diff --git a/zipkin/src/main/java/zipkin2/internal/Proto3ZipkinFields.java b/zipkin/src/main/java/zipkin2/internal/Proto3ZipkinFields.java
index a1731d8..7bc63a9 100644
--- a/zipkin/src/main/java/zipkin2/internal/Proto3ZipkinFields.java
+++ b/zipkin/src/main/java/zipkin2/internal/Proto3ZipkinFields.java
@@ -69,14 +69,14 @@ final class Proto3ZipkinFields {
       return result;
     }
 
-    @Override void writeValue(Buffer b, Endpoint value) {
+    @Override void writeValue(UnsafeBuffer b, Endpoint value) {
       SERVICE_NAME.write(b, value.serviceName());
       IPV4.write(b, value.ipv4Bytes());
       IPV6.write(b, value.ipv6Bytes());
       PORT.write(b, value.portAsInt());
     }
 
-    @Override Endpoint readValue(Buffer buffer, int length) {
+    @Override Endpoint readValue(UnsafeBuffer buffer, int length) {
       int endPos = buffer.pos() + length;
 
       // now, we are in the endpoint fields
@@ -111,11 +111,11 @@ final class Proto3ZipkinFields {
       super(key);
     }
 
-    @Override final T readValue(Buffer b, int length) {
+    @Override final T readValue(UnsafeBuffer b, int length) {
       throw new UnsupportedOperationException();
     }
 
-    abstract boolean readLengthPrefixAndValue(Buffer b, Span.Builder builder);
+    abstract boolean readLengthPrefixAndValue(UnsafeBuffer b, Span.Builder builder);
   }
 
   static class AnnotationField extends SpanBuilderField<Annotation> {
@@ -133,12 +133,12 @@ final class Proto3ZipkinFields {
       return TIMESTAMP.sizeInBytes(value.timestamp()) + VALUE.sizeInBytes(value.value());
     }
 
-    @Override void writeValue(Buffer b, Annotation value) {
+    @Override void writeValue(UnsafeBuffer b, Annotation value) {
       TIMESTAMP.write(b, value.timestamp());
       VALUE.write(b, value.value());
     }
 
-    @Override boolean readLengthPrefixAndValue(Buffer b, Span.Builder builder) {
+    @Override boolean readLengthPrefixAndValue(UnsafeBuffer b, Span.Builder builder) {
       int length = readLengthPrefix(b);
       if (length == 0) return false;
       int endPos = b.pos() + length;
@@ -181,12 +181,12 @@ final class Proto3ZipkinFields {
       return KEY.sizeInBytes(value.getKey()) + VALUE.sizeInBytes(value.getValue());
     }
 
-    @Override void writeValue(Buffer b, Map.Entry<String, String> value) {
+    @Override void writeValue(UnsafeBuffer b, Map.Entry<String, String> value) {
       KEY.write(b, value.getKey());
       VALUE.write(b, value.getValue());
     }
 
-    @Override boolean readLengthPrefixAndValue(Buffer b, Span.Builder builder) {
+    @Override boolean readLengthPrefixAndValue(UnsafeBuffer b, Span.Builder builder) {
       int length = readLengthPrefix(b);
       if (length == 0) return false;
       int endPos = b.pos() + length;
@@ -276,7 +276,7 @@ final class Proto3ZipkinFields {
       return sizeOfSpan;
     }
 
-    @Override void writeValue(Buffer b, Span value) {
+    @Override void writeValue(UnsafeBuffer b, Span value) {
       TRACE_ID.write(b, value.traceId());
       PARENT_ID.write(b, value.parentId());
       ID.write(b, value.id());
@@ -309,12 +309,12 @@ final class Proto3ZipkinFields {
       return kind != null ? kind.ordinal() + 1 : 0;
     }
 
-    public Span read(Buffer buffer) {
+    public Span read(UnsafeBuffer buffer) {
       buffer.readVarint32(); // toss the key
       return readLengthPrefixAndValue(buffer);
     }
 
-    @Override Span readValue(Buffer buffer, int length) {
+    @Override Span readValue(UnsafeBuffer buffer, int length) {
       int endPos = buffer.pos() + length;
 
       // now, we are in the span fields
@@ -372,7 +372,7 @@ final class Proto3ZipkinFields {
     }
   }
 
-  static void logAndSkip(Buffer buffer, int nextKey) {
+  static void logAndSkip(UnsafeBuffer buffer, int nextKey) {
     int nextWireType = wireType(nextKey, buffer.pos());
     if (LOG.isLoggable(FINE)) {
       int nextFieldNumber = fieldNumber(nextKey, buffer.pos());
diff --git a/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java b/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java
index ef38c5d..2355b4d 100644
--- a/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java
+++ b/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java
@@ -27,7 +27,6 @@ import zipkin2.Span;
 import zipkin2.v1.V1Span;
 import zipkin2.v1.V1SpanConverter;
 
-import static zipkin2.internal.Buffer.utf8SizeInBytes;
 import static zipkin2.internal.ThriftField.TYPE_BOOL;
 import static zipkin2.internal.ThriftField.TYPE_BYTE;
 import static zipkin2.internal.ThriftField.TYPE_DOUBLE;
@@ -40,6 +39,7 @@ import static zipkin2.internal.ThriftField.TYPE_SET;
 import static zipkin2.internal.ThriftField.TYPE_STOP;
 import static zipkin2.internal.ThriftField.TYPE_STRING;
 import static zipkin2.internal.ThriftField.TYPE_STRUCT;
+import static zipkin2.internal.UnsafeBuffer.utf8SizeInBytes;
 
 // @Immutable
 public final class ThriftCodec {
@@ -58,7 +58,7 @@ public final class ThriftCodec {
   }
 
   /** Encoding overhead is thrift type plus 32-bit length prefix */
-  static <T> int listSizeInBytes(Buffer.Writer<T> writer, List<T> values) {
+  static <T> int listSizeInBytes(UnsafeBuffer.Writer<T> writer, List<T> values) {
     int sizeInBytes = 5;
     for (int i = 0, length = values.size(); i < length; i++) {
       sizeInBytes += writer.sizeInBytes(values.get(i));
@@ -116,7 +116,7 @@ public final class ThriftCodec {
     return guardLength(bytes);
   }
 
-  static <T> void writeList(Buffer.Writer<T> writer, List<T> value, Buffer buffer) {
+  static <T> void writeList(UnsafeBuffer.Writer<T> writer, List<T> value, UnsafeBuffer buffer) {
     int length = value.size();
     writeListBegin(buffer, length);
     for (int i = 0; i < length; i++) {
@@ -212,25 +212,25 @@ public final class ThriftCodec {
     return length;
   }
 
-  static void writeListBegin(Buffer buffer, int size) {
+  static void writeListBegin(UnsafeBuffer buffer, int size) {
     buffer.writeByte(TYPE_STRUCT);
     writeInt(buffer, size);
   }
 
-  static void writeLengthPrefixed(Buffer buffer, String utf8) {
+  static void writeLengthPrefixed(UnsafeBuffer buffer, String utf8) {
     int ignoredLength = utf8SizeInBytes(utf8);
     writeInt(buffer, utf8SizeInBytes(utf8));
     buffer.writeUtf8(utf8);
   }
 
-  static void writeInt(Buffer buf, int v) {
+  static void writeInt(UnsafeBuffer buf, int v) {
     buf.writeByte((byte) ((v >>> 24L) & 0xff));
     buf.writeByte((byte) ((v >>> 16L) & 0xff));
     buf.writeByte((byte) ((v >>> 8L) & 0xff));
     buf.writeByte((byte) (v & 0xff));
   }
 
-  static void writeLong(Buffer buf, long v) {
+  static void writeLong(UnsafeBuffer buf, long v) {
     buf.writeByte((byte) ((v >>> 56L) & 0xff));
     buf.writeByte((byte) ((v >>> 48L) & 0xff));
     buf.writeByte((byte) ((v >>> 40L) & 0xff));
diff --git a/zipkin/src/main/java/zipkin2/internal/ThriftEndpointCodec.java b/zipkin/src/main/java/zipkin2/internal/ThriftEndpointCodec.java
index 5df70e1..9c0eb93 100644
--- a/zipkin/src/main/java/zipkin2/internal/ThriftEndpointCodec.java
+++ b/zipkin/src/main/java/zipkin2/internal/ThriftEndpointCodec.java
@@ -19,12 +19,12 @@ package zipkin2.internal;
 import java.nio.ByteBuffer;
 import zipkin2.Endpoint;
 
-import static zipkin2.internal.Buffer.utf8SizeInBytes;
 import static zipkin2.internal.ThriftCodec.skip;
 import static zipkin2.internal.ThriftField.TYPE_I16;
 import static zipkin2.internal.ThriftField.TYPE_I32;
 import static zipkin2.internal.ThriftField.TYPE_STOP;
 import static zipkin2.internal.ThriftField.TYPE_STRING;
+import static zipkin2.internal.UnsafeBuffer.utf8SizeInBytes;
 
 final class ThriftEndpointCodec {
   static final byte[] INT_ZERO = {0, 0, 0, 0};
@@ -75,7 +75,7 @@ final class ThriftEndpointCodec {
     return sizeInBytes;
   }
 
-  static void write(Endpoint value, Buffer buffer) {
+  static void write(Endpoint value, UnsafeBuffer buffer) {
     IPV4.write(buffer);
     buffer.write(value.ipv4Bytes() != null ? value.ipv4Bytes() : INT_ZERO);
 
diff --git a/zipkin/src/main/java/zipkin2/internal/ThriftField.java b/zipkin/src/main/java/zipkin2/internal/ThriftField.java
index 424e0c2..e0d62b4 100644
--- a/zipkin/src/main/java/zipkin2/internal/ThriftField.java
+++ b/zipkin/src/main/java/zipkin2/internal/ThriftField.java
@@ -41,7 +41,7 @@ final class ThriftField {
     this.id = id;
   }
 
-  void write(Buffer buffer) {
+  void write(UnsafeBuffer buffer) {
     buffer.writeByte(type);
     // Write ID as a short!
     buffer.writeByte((id >>> 8L) & 0xff);
diff --git a/zipkin/src/main/java/zipkin2/internal/Buffer.java b/zipkin/src/main/java/zipkin2/internal/UnsafeBuffer.java
similarity index 69%
rename from zipkin/src/main/java/zipkin2/internal/Buffer.java
rename to zipkin/src/main/java/zipkin2/internal/UnsafeBuffer.java
index c5d9b8b..8f32cd8 100644
--- a/zipkin/src/main/java/zipkin2/internal/Buffer.java
+++ b/zipkin/src/main/java/zipkin2/internal/UnsafeBuffer.java
@@ -16,21 +16,135 @@
  */
 package zipkin2.internal;
 
-public abstract class Buffer {
+import static zipkin2.internal.JsonCodec.UTF_8;
 
-  public static Buffer wrap(byte[] bytes, int pos) {
-    return new ByteArrayBuffer(bytes, pos);
+/**
+ * <p>Read operations do bounds checks, as typically more errors occur reading than writing.
+ *
+ * <p>Writes are unsafe as they do no bounds checks. This means you should take care to allocate or
+ * wrap an array at least as big as you need prior to writing. As it is possible to calculate size
+ * prior to writing, overrunning a buffer is a programming error.
+ */
+public final class UnsafeBuffer {
+
+  public static UnsafeBuffer wrap(byte[] bytes, int pos) {
+    return new UnsafeBuffer(bytes, pos);
   }
 
-  public static Buffer allocate(int sizeInBytes) {
-    return new ByteArrayBuffer(sizeInBytes);
+  public static UnsafeBuffer allocate(int sizeInBytes) {
+    return new UnsafeBuffer(sizeInBytes);
   }
 
-  static final byte[] DIGITS = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9'};
   static final char[] HEX_DIGITS = {
     '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'
   };
 
+  private final byte[] buf;
+  int pos; // visible for testing
+
+  UnsafeBuffer(int size) {
+    buf = new byte[size];
+  }
+
+  UnsafeBuffer(byte[] buf, int pos) {
+    this.buf = buf;
+    this.pos = pos;
+  }
+
+  public void writeByte(int v) {
+    buf[pos++] = (byte) (v & 0xff);
+  }
+
+  public void write(byte[] v) {
+    System.arraycopy(v, 0, buf, pos, v.length);
+    pos += v.length;
+  }
+
+  void writeBackwards(long v) {
+    int lastPos = pos + asciiSizeInBytes(v); // We write backwards from right to left.
+    pos = lastPos;
+    while (v != 0) {
+      int digit = (int) (v % 10);
+      buf[--lastPos] = (byte) HEX_DIGITS[digit];
+      v /= 10;
+    }
+  }
+
+  /** Inspired by {@code okio.Buffer.writeLong} */
+  public void writeLongHex(long v) {
+    int pos = this.pos;
+    writeHexByte(buf, pos + 0, (byte) ((v >>> 56L) & 0xff));
+    writeHexByte(buf, pos + 2, (byte) ((v >>> 48L) & 0xff));
+    writeHexByte(buf, pos + 4, (byte) ((v >>> 40L) & 0xff));
+    writeHexByte(buf, pos + 6, (byte) ((v >>> 32L) & 0xff));
+    writeHexByte(buf, pos + 8, (byte) ((v >>> 24L) & 0xff));
+    writeHexByte(buf, pos + 10, (byte) ((v >>> 16L) & 0xff));
+    writeHexByte(buf, pos + 12, (byte) ((v >>> 8L) & 0xff));
+    writeHexByte(buf, pos + 14, (byte) (v & 0xff));
+    this.pos = pos + 16;
+  }
+
+  // reset for reading
+  public void reset() {
+    pos = 0;
+  }
+
+  byte[] readBytes(int length) {
+    require(length);
+    byte[] result = new byte[length];
+    System.arraycopy(buf, pos, result, 0, length);
+    pos += length;
+    return result;
+  }
+
+  String readUtf8(int length) {
+    require(length);
+    String result = new String(buf, pos, length, UTF_8);
+    pos += length;
+    return result;
+  }
+
+  String readBytesAsHex(int length) {
+    // All our hex fields are at most 32 characters.
+    if (length > 32) {
+      throw new IllegalArgumentException("hex field greater than 32 chars long: " + length);
+    }
+
+    require(length);
+    char[] result = Platform.get().idBuffer();
+
+    int hexLength = length * 2;
+    for (int i = 0; i < hexLength; i += 2) {
+      byte b = buf[pos++];
+      result[i + 0] = HEX_DIGITS[(b >> 4) & 0xf];
+      result[i + 1] = HEX_DIGITS[b & 0xf];
+    }
+    return new String(result, 0, hexLength);
+  }
+
+  int remaining() {
+    return buf.length - pos;
+  }
+
+  boolean skip(int maxCount) {
+    int nextPos = pos + maxCount;
+    if (nextPos > buf.length) {
+      pos = buf.length;
+      return false;
+    }
+    pos = nextPos;
+    return true;
+  }
+
+  public int pos() {
+    return pos;
+  }
+
+  public byte[] unwrap() {
+    // assert pos == buf.length;
+    return buf;
+  }
+
   /**
    * This returns the bytes needed to transcode a UTF-16 Java String to UTF-8 bytes.
    *
@@ -135,13 +249,9 @@ public abstract class Buffer {
     data[pos + 1] = (byte) HEX_DIGITS[b & 0xf];
   }
 
-  public abstract void writeByte(int v);
-
-  public abstract void write(byte[] v);
-
   public void writeAscii(String v) {
-    for (int i = 0, len = v.length(); i < len; i++) {
-      writeByte(v.charAt(i));
+    for (int i = 0, length = v.length(); i < length; i++) {
+      writeByte(v.charAt(i) & 0xff);
     }
   }
 
@@ -199,6 +309,7 @@ public abstract class Buffer {
   // Adapted from okio.Buffer.writeDecimalLong
   public void writeAscii(long v) {
     if (v == 0) {
+      require(1);
       writeByte('0');
       return;
     }
@@ -216,8 +327,6 @@ public abstract class Buffer {
     writeBackwards(v);
   }
 
-  abstract void writeBackwards(long v);
-
   // com.squareup.wire.ProtoWriter.writeVarint v2.3.0
   void writeVarint(int v) {
     while ((v & ~0x7f) != 0) {
@@ -236,8 +345,6 @@ public abstract class Buffer {
     writeByte((byte) v);
   }
 
-  public abstract void writeLongHex(long v);
-
   void writeLongLe(long v) {
     writeByte((byte) (v & 0xff));
     writeByte((byte) ((v >> 8) & 0xff));
@@ -249,33 +356,27 @@ public abstract class Buffer {
     writeByte((byte) ((v >> 56) & 0xff));
   }
 
-  // reset for reading
-  public abstract void reset();
-
   long readLongLe() {
-    ensureLength(this, 8);
-    return (readByteUnsafe() & 0xffL)
-      | (readByteUnsafe() & 0xffL) << 8
-      | (readByteUnsafe() & 0xffL) << 16
-      | (readByteUnsafe() & 0xffL) << 24
-      | (readByteUnsafe() & 0xffL) << 32
-      | (readByteUnsafe() & 0xffL) << 40
-      | (readByteUnsafe() & 0xffL) << 48
-      | (readByteUnsafe() & 0xffL) << 56;
+    require(8);
+    int pos = this.pos;
+    this.pos = pos + 8;
+    return (buf[pos] & 0xffL)
+      | (buf[pos + 1] & 0xffL) << 8
+      | (buf[pos + 2] & 0xffL) << 16
+      | (buf[pos + 3] & 0xffL) << 24
+      | (buf[pos + 4] & 0xffL) << 32
+      | (buf[pos + 5] & 0xffL) << 40
+      | (buf[pos + 6] & 0xffL) << 48
+      | (buf[pos + 7] & 0xffL) << 56;
   }
 
   final byte readByte() {
-    ensureLength(this, 1);
-    return readByteUnsafe();
+    require(1);
+    return buf[pos++];
   }
 
-  /** This needs to be checked externally to not overrun the underlying array */
-  abstract byte readByteUnsafe();
-
-  abstract byte[] readByteArray(int byteCount);
-
   /**
-   * @return the value read. Use {@link Buffer#varintSizeInBytes(long)} to tell how many bytes.
+   * @return the value read. Use {@link #varintSizeInBytes(long)} to tell how many bytes.
    * @throws IllegalArgumentException if more than 64 bits were encoded
    */
   // included in the main api as this is used commonly, for example reading proto tags
@@ -303,7 +404,7 @@ public abstract class Buffer {
 
     b = readByte();
     if ((b & 0xf0) != 0) {
-      throw new IllegalArgumentException("Greater than 32-bit varint at position " + (pos() - 1));
+      throw new IllegalArgumentException("Greater than 32-bit varint at position " + (pos - 1));
     }
     return result | b << 28;
   }
@@ -318,31 +419,23 @@ public abstract class Buffer {
     for (int i = 1; b < 0 && i < 10; i++) {
       b = readByte();
       if (i == 9 && (b & 0xf0) != 0) {
-        throw new IllegalArgumentException("Greater than 64-bit varint at position " + (pos() - 1));
+        throw new IllegalArgumentException("Greater than 64-bit varint at position " + (pos - 1));
       }
       result |= (long) (b & 0x7f) << (i * 7);
     }
     return result;
   }
 
-  abstract int remaining();
-
-  abstract boolean skip(int maxCount);
-
-  public abstract int pos();
-
-  public abstract byte[] toByteArrayUnsafe();
-
   public interface Writer<T> {
     int sizeInBytes(T value);
 
-    void write(T value, Buffer buffer);
+    void write(T value, UnsafeBuffer buffer);
   }
 
-  static void ensureLength(Buffer buffer, int length) {
-    if (length > buffer.remaining()) {
+  void require(int byteCount) {
+    if (pos + byteCount > buf.length) {
       throw new IllegalArgumentException(
-        "Truncated: length " + length + " > bytes remaining " + buffer.remaining());
+        "Truncated: length " + byteCount + " > bytes remaining " + remaining());
     }
   }
 }
diff --git a/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java b/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java
index 76a6388..d16c7e1 100644
--- a/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java
+++ b/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java
@@ -22,7 +22,7 @@ import zipkin2.v1.V2SpanConverter;
 
 /** This type isn't thread-safe: it re-uses state to avoid re-allocations in conversion loops. */
 // @Immutable
-public final class V1JsonSpanWriter implements Buffer.Writer<Span> {
+public final class V1JsonSpanWriter implements UnsafeBuffer.Writer<Span> {
   final V2SpanConverter converter = V2SpanConverter.create();
   final V1SpanWriter v1SpanWriter = new V1SpanWriter();
 
@@ -33,7 +33,7 @@ public final class V1JsonSpanWriter implements Buffer.Writer<Span> {
   }
 
   @Override
-  public void write(Span value, Buffer b) {
+  public void write(Span value, UnsafeBuffer b) {
     V1Span v1Span = converter.convert(value);
     v1SpanWriter.write(v1Span, b);
   }
diff --git a/zipkin/src/main/java/zipkin2/internal/V1SpanWriter.java b/zipkin/src/main/java/zipkin2/internal/V1SpanWriter.java
index 68070d9..d891ca3 100644
--- a/zipkin/src/main/java/zipkin2/internal/V1SpanWriter.java
+++ b/zipkin/src/main/java/zipkin2/internal/V1SpanWriter.java
@@ -21,7 +21,7 @@ import zipkin2.v1.V1Annotation;
 import zipkin2.v1.V1BinaryAnnotation;
 import zipkin2.v1.V1Span;
 
-import static zipkin2.internal.Buffer.asciiSizeInBytes;
+import static zipkin2.internal.UnsafeBuffer.asciiSizeInBytes;
 import static zipkin2.internal.JsonEscaper.jsonEscape;
 import static zipkin2.internal.JsonEscaper.jsonEscapedSizeInBytes;
 import static zipkin2.internal.V2SpanWriter.endpointSizeInBytes;
@@ -29,7 +29,7 @@ import static zipkin2.internal.V2SpanWriter.writeAnnotation;
 
 /** This type is only used to backport the v1 read api as it returns v1 json. */
 // @Immutable
-public final class V1SpanWriter implements Buffer.Writer<V1Span> {
+public final class V1SpanWriter implements UnsafeBuffer.Writer<V1Span> {
 
   @Override
   public int sizeInBytes(V1Span value) {
@@ -104,7 +104,7 @@ public final class V1SpanWriter implements Buffer.Writer<V1Span> {
   }
 
   @Override
-  public void write(V1Span value, Buffer b) {
+  public void write(V1Span value, UnsafeBuffer b) {
     b.writeAscii("{\"traceId\":\"");
     if (value.traceIdHigh() != 0L) b.writeLongHex(value.traceIdHigh());
     b.writeLongHex(value.traceId());
@@ -193,9 +193,9 @@ public final class V1SpanWriter implements Buffer.Writer<V1Span> {
 
   static byte[] legacyEndpointBytes(@Nullable Endpoint localEndpoint) {
     if (localEndpoint == null) return null;
-    Buffer buffer = Buffer.allocate(endpointSizeInBytes(localEndpoint, true));
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(endpointSizeInBytes(localEndpoint, true));
     V2SpanWriter.writeEndpoint(localEndpoint, buffer, true);
-    return buffer.toByteArrayUnsafe();
+    return buffer.unwrap();
   }
 
   static int binaryAnnotationSizeInBytes(String key, String value, int endpointSize) {
@@ -209,7 +209,7 @@ public final class V1SpanWriter implements Buffer.Writer<V1Span> {
     return sizeInBytes;
   }
 
-  static void writeBinaryAnnotation(String key, String value, @Nullable byte[] endpoint, Buffer b) {
+  static void writeBinaryAnnotation(String key, String value, @Nullable byte[] endpoint, UnsafeBuffer b) {
     b.writeAscii("{\"key\":\"");
     b.writeUtf8(jsonEscape(key));
     b.writeAscii("\",\"value\":\"");
diff --git a/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java b/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java
index d3c570c..5c80d1c 100644
--- a/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java
+++ b/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java
@@ -24,7 +24,7 @@ import zipkin2.v1.V1BinaryAnnotation;
 import zipkin2.v1.V1Span;
 import zipkin2.v1.V2SpanConverter;
 
-import static zipkin2.internal.Buffer.utf8SizeInBytes;
+import static zipkin2.internal.UnsafeBuffer.utf8SizeInBytes;
 import static zipkin2.internal.ThriftField.TYPE_BOOL;
 import static zipkin2.internal.ThriftField.TYPE_I32;
 import static zipkin2.internal.ThriftField.TYPE_I64;
@@ -35,7 +35,7 @@ import static zipkin2.internal.ThriftField.TYPE_STRUCT;
 
 /** This type isn't thread-safe: it re-uses state to avoid re-allocations in conversion loops. */
 // @Immutable
-public final class V1ThriftSpanWriter implements Buffer.Writer<Span> {
+public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
 
   static final ThriftField TRACE_ID = new ThriftField(TYPE_I64, 1);
   static final ThriftField TRACE_ID_HIGH = new ThriftField(TYPE_I64, 12);
@@ -95,7 +95,7 @@ public final class V1ThriftSpanWriter implements Buffer.Writer<Span> {
   }
 
   @Override
-  public void write(Span value, Buffer buffer) {
+  public void write(Span value, UnsafeBuffer buffer) {
     V1Span v1Span = converter.convert(value);
     byte[] endpointBytes = legacyEndpointBytes(value.localEndpoint());
 
@@ -142,7 +142,7 @@ public final class V1ThriftSpanWriter implements Buffer.Writer<Span> {
     buffer.writeByte(TYPE_STOP);
   }
 
-  static void writeAnnotations(Buffer buffer, V1Span v1Span, byte[] endpointBytes) {
+  static void writeAnnotations(UnsafeBuffer buffer, V1Span v1Span, byte[] endpointBytes) {
     int annotationCount = v1Span.annotations().size();
     ThriftCodec.writeListBegin(buffer, annotationCount);
     for (int i = 0; i < annotationCount; i++) {
@@ -151,7 +151,7 @@ public final class V1ThriftSpanWriter implements Buffer.Writer<Span> {
     }
   }
 
-  static void writeBinaryAnnotations(Buffer buffer, V1Span v1Span, byte[] endpointBytes) {
+  static void writeBinaryAnnotations(UnsafeBuffer buffer, V1Span v1Span, byte[] endpointBytes) {
     int binaryAnnotationCount = v1Span.binaryAnnotations().size();
     ThriftCodec.writeListBegin(buffer, binaryAnnotationCount);
     for (int i = 0; i < binaryAnnotationCount; i++) {
@@ -170,22 +170,22 @@ public final class V1ThriftSpanWriter implements Buffer.Writer<Span> {
     int lengthOfSpans = spans.size();
     if (lengthOfSpans == 0) return EMPTY_ARRAY;
 
-    Buffer result = Buffer.allocate(ThriftCodec.listSizeInBytes(this, spans));
+    UnsafeBuffer result = UnsafeBuffer.allocate(ThriftCodec.listSizeInBytes(this, spans));
     ThriftCodec.writeList(this, spans, result);
-    return result.toByteArrayUnsafe();
+    return result.unwrap();
   }
 
   public byte[] write(Span onlySpan) {
-    Buffer result = Buffer.allocate(sizeInBytes(onlySpan));
+    UnsafeBuffer result = UnsafeBuffer.allocate(sizeInBytes(onlySpan));
     write(onlySpan, result);
-    return result.toByteArrayUnsafe();
+    return result.unwrap();
   }
 
   public int writeList(List<Span> spans, byte[] out, int pos) {
     int lengthOfSpans = spans.size();
     if (lengthOfSpans == 0) return 0;
 
-    Buffer result = Buffer.wrap(out, pos);
+    UnsafeBuffer result = UnsafeBuffer.wrap(out, pos);
     ThriftCodec.writeList(this, spans, result);
 
     return result.pos() - pos;
@@ -193,9 +193,9 @@ public final class V1ThriftSpanWriter implements Buffer.Writer<Span> {
 
   static byte[] legacyEndpointBytes(@Nullable Endpoint localEndpoint) {
     if (localEndpoint == null) return null;
-    Buffer buffer = Buffer.allocate(ThriftEndpointCodec.sizeInBytes(localEndpoint));
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(ThriftEndpointCodec.sizeInBytes(localEndpoint));
     ThriftEndpointCodec.write(localEndpoint, buffer);
-    return buffer.toByteArrayUnsafe();
+    return buffer.unwrap();
   }
 
   static class ThriftAnnotationWriter {
@@ -213,7 +213,7 @@ public final class V1ThriftSpanWriter implements Buffer.Writer<Span> {
       return sizeInBytes;
     }
 
-    static void write(long timestamp, String value, byte[] endpointBytes, Buffer buffer) {
+    static void write(long timestamp, String value, byte[] endpointBytes, UnsafeBuffer buffer) {
       TIMESTAMP.write(buffer);
       ThriftCodec.writeLong(buffer, timestamp);
 
@@ -245,7 +245,7 @@ public final class V1ThriftSpanWriter implements Buffer.Writer<Span> {
       return sizeInBytes;
     }
 
-    static void write(String key, String stringValue, byte[] endpointBytes, Buffer buffer) {
+    static void write(String key, String stringValue, byte[] endpointBytes, UnsafeBuffer buffer) {
       KEY.write(buffer);
       ThriftCodec.writeLengthPrefixed(buffer, key);
 
diff --git a/zipkin/src/main/java/zipkin2/internal/V2SpanWriter.java b/zipkin/src/main/java/zipkin2/internal/V2SpanWriter.java
index 9839533..9dbba53 100644
--- a/zipkin/src/main/java/zipkin2/internal/V2SpanWriter.java
+++ b/zipkin/src/main/java/zipkin2/internal/V2SpanWriter.java
@@ -22,12 +22,12 @@ import zipkin2.Annotation;
 import zipkin2.Endpoint;
 import zipkin2.Span;
 
-import static zipkin2.internal.Buffer.asciiSizeInBytes;
+import static zipkin2.internal.UnsafeBuffer.asciiSizeInBytes;
 import static zipkin2.internal.JsonEscaper.jsonEscape;
 import static zipkin2.internal.JsonEscaper.jsonEscapedSizeInBytes;
 
 // @Immutable
-public final class V2SpanWriter implements Buffer.Writer<Span> {
+public final class V2SpanWriter implements UnsafeBuffer.Writer<Span> {
   @Override
   public int sizeInBytes(Span value) {
     int sizeInBytes = 13; // {"traceId":""
@@ -89,7 +89,7 @@ public final class V2SpanWriter implements Buffer.Writer<Span> {
   }
 
   @Override
-  public void write(Span value, Buffer b) {
+  public void write(Span value, UnsafeBuffer b) {
     b.writeAscii("{\"traceId\":\"");
     b.writeAscii(value.traceId());
     b.writeByte('"');
@@ -192,7 +192,7 @@ public final class V2SpanWriter implements Buffer.Writer<Span> {
     return ++sizeInBytes; // }
   }
 
-  static void writeEndpoint(Endpoint value, Buffer b, boolean writeEmptyServiceName) {
+  static void writeEndpoint(Endpoint value, UnsafeBuffer b, boolean writeEmptyServiceName) {
     b.writeByte('{');
     boolean wroteField = false;
     String serviceName = value.serviceName();
@@ -237,7 +237,7 @@ public final class V2SpanWriter implements Buffer.Writer<Span> {
     return sizeInBytes;
   }
 
-  static void writeAnnotation(long timestamp, String value, @Nullable byte[] endpoint, Buffer b) {
+  static void writeAnnotation(long timestamp, String value, @Nullable byte[] endpoint, UnsafeBuffer b) {
     b.writeAscii("{\"timestamp\":");
     b.writeAscii(timestamp);
     b.writeAscii(",\"value\":\"");
diff --git a/zipkin/src/test/java/zipkin2/internal/JsonCodecTest.java b/zipkin/src/test/java/zipkin2/internal/JsonCodecTest.java
index af8d253..5421518 100644
--- a/zipkin/src/test/java/zipkin2/internal/JsonCodecTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/JsonCodecTest.java
@@ -17,7 +17,6 @@
 package zipkin2.internal;
 
 import java.io.IOException;
-import org.assertj.core.api.Assertions;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -34,12 +33,12 @@ public class JsonCodecTest {
     thrown.expect(AssertionError.class);
     thrown.expectMessage("Bug found using FooWriter to write Foo as json. Wrote 1/2 bytes: a");
 
-    class FooWriter implements Buffer.Writer {
+    class FooWriter implements UnsafeBuffer.Writer {
       @Override public int sizeInBytes(Object value) {
         return 2;
       }
 
-      @Override public void write(Object value, Buffer buffer) {
+      @Override public void write(Object value, UnsafeBuffer buffer) {
         buffer.writeByte('a');
         throw new RuntimeException("buggy");
       }
@@ -59,12 +58,12 @@ public class JsonCodecTest {
     thrown.expectMessage("Bug found using FooWriter to write Foo as json. Wrote 2/2 bytes: ab");
 
     // pretend there was a bug calculating size, ex it calculated incorrectly as to small
-    class FooWriter implements Buffer.Writer {
+    class FooWriter implements UnsafeBuffer.Writer {
       @Override public int sizeInBytes(Object value) {
         return 2;
       }
 
-      @Override public void write(Object value, Buffer buffer) {
+      @Override public void write(Object value, UnsafeBuffer buffer) {
         buffer.writeByte('a');
         buffer.writeByte('b');
         buffer.writeByte('c'); // wrote larger than size!
diff --git a/zipkin/src/test/java/zipkin2/internal/Proto3FieldsTest.java b/zipkin/src/test/java/zipkin2/internal/Proto3FieldsTest.java
index 5a4882e..5e1cef9 100644
--- a/zipkin/src/test/java/zipkin2/internal/Proto3FieldsTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/Proto3FieldsTest.java
@@ -35,7 +35,7 @@ import static zipkin2.internal.Proto3Fields.WIRETYPE_LENGTH_DELIMITED;
 import static zipkin2.internal.Proto3Fields.WIRETYPE_VARINT;
 
 public class Proto3FieldsTest {
-  Buffer buf = Buffer.allocate(2048); // bigger than needed to test sizeOf
+  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
 
   /** Shows we can reliably look at a byte zero to tell if we are decoding proto3 repeated fields. */
   @Test public void field_key_fieldOneLengthDelimited() {
@@ -185,7 +185,7 @@ public class Proto3FieldsTest {
 
   @Test public void field_readLengthPrefixAndValue_LENGTH_DELIMITED_truncated() {
     BytesField field = new BytesField(128 << 3 | WIRETYPE_LENGTH_DELIMITED);
-    buf = Buffer.allocate(10);
+    buf = UnsafeBuffer.allocate(10);
     buf.writeVarint(100); // much larger than the buffer size
     buf.reset();
 
diff --git a/zipkin/src/test/java/zipkin2/internal/Proto3SpanWriterTest.java b/zipkin/src/test/java/zipkin2/internal/Proto3SpanWriterTest.java
index 7e5708c..03c31de 100644
--- a/zipkin/src/test/java/zipkin2/internal/Proto3SpanWriterTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/Proto3SpanWriterTest.java
@@ -24,7 +24,7 @@ import static zipkin2.TestObjects.CLIENT_SPAN;
 import static zipkin2.internal.Proto3ZipkinFields.SPAN;
 
 public class Proto3SpanWriterTest {
-  Buffer buf = Buffer.allocate(2048); // bigger than needed to test sizeOf
+  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
 
   Proto3SpanWriter writer = new Proto3SpanWriter();
 
@@ -59,9 +59,9 @@ public class Proto3SpanWriterTest {
   }
 
   @Test public void writeList_offset_startsWithSpanKeyAndLengthPrefix() {
-    writer.writeList(asList(CLIENT_SPAN, CLIENT_SPAN), buf.toByteArrayUnsafe(), 0);
+    writer.writeList(asList(CLIENT_SPAN, CLIENT_SPAN), buf.unwrap(), 0);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
       .startsWith((byte) 10, SPAN.sizeOfValue(CLIENT_SPAN));
   }
 }
diff --git a/zipkin/src/test/java/zipkin2/internal/Proto3ZipkinFieldsTest.java b/zipkin/src/test/java/zipkin2/internal/Proto3ZipkinFieldsTest.java
index 37b1e81..ad3d139 100644
--- a/zipkin/src/test/java/zipkin2/internal/Proto3ZipkinFieldsTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/Proto3ZipkinFieldsTest.java
@@ -35,7 +35,7 @@ import static zipkin2.internal.Proto3Fields.WIRETYPE_LENGTH_DELIMITED;
 import static zipkin2.internal.Proto3ZipkinFields.SPAN;
 
 public class Proto3ZipkinFieldsTest {
-  Buffer buf = Buffer.allocate(2048); // bigger than needed to test sizeInBytes
+  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeInBytes
 
   /** A map entry is an embedded messages: one for field the key and one for the value */
   @Test public void tag_sizeInBytes() {
@@ -79,14 +79,14 @@ public class Proto3ZipkinFieldsTest {
   @Test public void span_write_startsWithFieldInListOfSpans() {
     SPAN.write(buf, spanBuilder().build());
 
-    assertThat(buf.toByteArrayUnsafe()).startsWith(
+    assertThat(buf.unwrap()).startsWith(
       0b00001010 /* span key */, 20 /* bytes for length of the span */
     );
   }
 
   @Test public void span_write_writesIds() {
     SPAN.write(buf, spanBuilder().build());
-    assertThat(buf.toByteArrayUnsafe()).startsWith(
+    assertThat(buf.unwrap()).startsWith(
       0b00001010 /* span key */, 20 /* bytes for length of the span */,
       0b00001010 /* trace ID key */, 8 /* bytes for 64-bit trace ID */,
       0, 0, 0, 0, 0, 0, 0, 1, // hex trace ID
@@ -154,7 +154,7 @@ public class Proto3ZipkinFieldsTest {
 
   @Test public void span_write_kind() {
     SPAN.write(buf, spanBuilder().kind(Span.Kind.PRODUCER).build());
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
       .contains(0b0100000, atIndex(22)) // (field_number << 3) | wire_type = 4 << 3 | 0
       .contains(0b0000011, atIndex(23)); // producer's index is 3
   }
@@ -163,12 +163,12 @@ public class Proto3ZipkinFieldsTest {
     assertRoundTrip(spanBuilder().kind(Span.Kind.CONSUMER).build());
 
     buf.reset();
-    buf.toByteArrayUnsafe()[23] = (byte) (Span.Kind.values().length + 1); // undefined kind
+    buf.unwrap()[23] = (byte) (Span.Kind.values().length + 1); // undefined kind
     assertThat(SPAN.read(buf))
       .isEqualTo(spanBuilder().build()); // skips undefined kind instead of dying
 
     buf.reset();
-    buf.toByteArrayUnsafe()[23] = 0; // serialized zero
+    buf.unwrap()[23] = 0; // serialized zero
     assertThat(SPAN.read(buf))
       .isEqualTo(spanBuilder().build());
   }
@@ -176,7 +176,7 @@ public class Proto3ZipkinFieldsTest {
   @Test public void span_write_debug() {
     SPAN.write(buf, CLIENT_SPAN.toBuilder().debug(true).build());
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
       .contains(0b01100000, atIndex(buf.pos() - 2)) // (field_number << 3) | wire_type = 12 << 3 | 0
       .contains(1, atIndex(buf.pos() - 1)); // true
   }
@@ -184,7 +184,7 @@ public class Proto3ZipkinFieldsTest {
   @Test public void span_write_shared() {
     SPAN.write(buf, CLIENT_SPAN.toBuilder().kind(Span.Kind.SERVER).shared(true).build());
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
       .contains(0b01101000, atIndex(buf.pos() - 2)) // (field_number << 3) | wire_type = 13 << 3 | 0
       .contains(1, atIndex(buf.pos() - 1)); // true
   }
diff --git a/zipkin/src/test/java/zipkin2/internal/BufferTest.java b/zipkin/src/test/java/zipkin2/internal/UnsafeBufferTest.java
similarity index 74%
rename from zipkin/src/test/java/zipkin2/internal/BufferTest.java
rename to zipkin/src/test/java/zipkin2/internal/UnsafeBufferTest.java
index ae3b869..22f97f2 100644
--- a/zipkin/src/test/java/zipkin2/internal/BufferTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/UnsafeBufferTest.java
@@ -26,7 +26,7 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown;
 import static zipkin2.TestObjects.UTF_8;
 
-public class BufferTest {
+public class UnsafeBufferTest {
   // Adapted from http://stackoverflow.com/questions/8511490/calculating-length-in-utf-8-of-java-string-without-actually-encoding-it
   @Test public void utf8SizeInBytes() {
     for (int codepoint = 0; codepoint <= 0x10FFFF; codepoint++) {
@@ -34,7 +34,7 @@ public class BufferTest {
       if (Character.isDefined(codepoint)) {
         String test = new String(Character.toChars(codepoint));
         int expected = test.getBytes(UTF_8).length;
-        int actual = Buffer.utf8SizeInBytes(test);
+        int actual = UnsafeBuffer.utf8SizeInBytes(test);
         if (actual != expected) {
           throw new AssertionError(actual + " length != " + expected + " for " + codepoint);
         }
@@ -46,12 +46,12 @@ public class BufferTest {
   @Test public void utf8_malformed() {
     for (int codepoint : Arrays.asList(0xD800, 0xDFFF, 0xD83D)) {
       String test = new String(new int[] {'a', codepoint, 'c'}, 0, 3);
-      assertThat(Buffer.utf8SizeInBytes(test))
+      assertThat(UnsafeBuffer.utf8SizeInBytes(test))
         .isEqualTo(3);
 
-      Buffer buffer = Buffer.allocate(3);
+      UnsafeBuffer buffer = UnsafeBuffer.allocate(3);
       buffer.writeUtf8(test);
-      assertThat(buffer.toByteArrayUnsafe())
+      assertThat(buffer.unwrap())
         .containsExactly('a', '?', 'c');
     }
   }
@@ -61,12 +61,12 @@ public class BufferTest {
     char[] array = "\uD83C\uDC00\uD83C\uDC01".toCharArray();
     array[array.length - 1] = 'c';
     String test = new String(array, 0, array.length - 1);
-    assertThat(Buffer.utf8SizeInBytes(test))
+    assertThat(UnsafeBuffer.utf8SizeInBytes(test))
       .isEqualTo(5);
 
-    Buffer buffer = Buffer.allocate(5);
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(5);
     buffer.writeUtf8(test);
-    assertThat(new String(buffer.toByteArrayUnsafe(), UTF_8))
+    assertThat(new String(buffer.unwrap(), UTF_8))
       .isEqualTo("\uD83C\uDC00?");
   }
 
@@ -75,12 +75,12 @@ public class BufferTest {
     char[] array = "\uD83C\uDC00\uD83C\uDC01".toCharArray();
     array[array.length - 1] = 'c';
     String test = new String(array);
-    assertThat(Buffer.utf8SizeInBytes(test))
+    assertThat(UnsafeBuffer.utf8SizeInBytes(test))
       .isEqualTo(6);
 
-    Buffer buffer = Buffer.allocate(6);
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(6);
     buffer.writeUtf8(test);
-    assertThat(new String(buffer.toByteArrayUnsafe(), UTF_8))
+    assertThat(new String(buffer.unwrap(), UTF_8))
       .isEqualTo("\uD83C\uDC00?c");
   }
 
@@ -91,43 +91,43 @@ public class BufferTest {
       "ю́ cyrillic small letter yu with acute",
       "∃y ∀x ¬(x ≺ y)"
     )) {
-      int encodedSize = Buffer.utf8SizeInBytes(string);
+      int encodedSize = UnsafeBuffer.utf8SizeInBytes(string);
       assertThat(encodedSize)
         .isEqualTo(string.getBytes(UTF_8).length);
 
-      Buffer bufferUtf8 = Buffer.allocate(encodedSize);
+      UnsafeBuffer bufferUtf8 = UnsafeBuffer.allocate(encodedSize);
       bufferUtf8.writeUtf8(string);
-      assertThat(new String(bufferUtf8.toByteArrayUnsafe(), UTF_8))
+      assertThat(new String(bufferUtf8.unwrap(), UTF_8))
         .isEqualTo(string);
     }
   }
 
   @Test public void utf8_matchesAscii() throws Exception {
     String ascii = "86154a4ba6e913854d1e00c0db9010db";
-    int encodedSize = Buffer.utf8SizeInBytes(ascii);
+    int encodedSize = UnsafeBuffer.utf8SizeInBytes(ascii);
     assertThat(encodedSize)
       .isEqualTo(ascii.length());
 
-    Buffer bufferAscii = Buffer.allocate(encodedSize);
+    UnsafeBuffer bufferAscii = UnsafeBuffer.allocate(encodedSize);
     bufferAscii.writeAscii(ascii);
-    assertThat(new String(bufferAscii.toByteArrayUnsafe(), "US-ASCII"))
+    assertThat(new String(bufferAscii.unwrap(), "US-ASCII"))
       .isEqualTo(ascii);
 
-    Buffer bufferUtf8 = Buffer.allocate(encodedSize);
+    UnsafeBuffer bufferUtf8 = UnsafeBuffer.allocate(encodedSize);
     bufferUtf8.writeUtf8(ascii);
-    assertThat(new String(bufferUtf8.toByteArrayUnsafe(), "US-ASCII"))
+    assertThat(new String(bufferUtf8.unwrap(), "US-ASCII"))
       .isEqualTo(ascii);
   }
 
   @Test public void emoji() {
     byte[] emojiBytes = {(byte) 0xF0, (byte) 0x9F, (byte) 0x98, (byte) 0x81};
     String emoji = new String(emojiBytes, UTF_8);
-    assertThat(Buffer.utf8SizeInBytes(emoji))
+    assertThat(UnsafeBuffer.utf8SizeInBytes(emoji))
       .isEqualTo(emojiBytes.length);
 
-    Buffer buffer = Buffer.allocate(emojiBytes.length);
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(emojiBytes.length);
     buffer.writeUtf8(emoji);
-    assertThat(buffer.toByteArrayUnsafe())
+    assertThat(buffer.unwrap())
       .isEqualTo(emojiBytes);
   }
 
@@ -143,9 +143,9 @@ public class BufferTest {
   }
 
   static String writeAscii(long v) {
-    Buffer buffer = Buffer.allocate(Buffer.asciiSizeInBytes(v));
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.asciiSizeInBytes(v));
     buffer.writeAscii(v);
-    return new String(buffer.toByteArrayUnsafe(), UTF_8);
+    return new String(buffer.unwrap(), UTF_8);
   }
 
   // Test creating Buffer for a long string
@@ -155,33 +155,33 @@ public class BufferTest {
       stringBuffer.append("a");
     }
     String string = stringBuffer.toString();
-    Buffer buffer = Buffer.allocate(string.length());
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(string.length());
     buffer.writeAscii(string);
-    assertThat(new String(buffer.toByteArrayUnsafe(), "US-ASCII")).isEqualTo(string);
+    assertThat(new String(buffer.unwrap(), "US-ASCII")).isEqualTo(string);
   }
 
   @Test public void unsignedVarintSize_32_largest() {
     // largest to encode is a negative number
-    assertThat(Buffer.varintSizeInBytes(Integer.MIN_VALUE))
+    assertThat(UnsafeBuffer.varintSizeInBytes(Integer.MIN_VALUE))
       .isEqualTo(5);
   }
 
   @Test public void unsignedVarintSize_64_largest() {
     // largest to encode is a negative number
-    assertThat(Buffer.varintSizeInBytes(Long.MIN_VALUE))
+    assertThat(UnsafeBuffer.varintSizeInBytes(Long.MIN_VALUE))
       .isEqualTo(10);
   }
 
   @Test public void writeLongLe_matchesByteBuffer() {
     for (long number : Arrays.asList(Long.MIN_VALUE, 0L, Long.MAX_VALUE)) {
-      Buffer buffer = Buffer.allocate(8);
+      UnsafeBuffer buffer = UnsafeBuffer.allocate(8);
       buffer.writeLongLe(number);
 
       ByteBuffer byteBuffer = ByteBuffer.allocate(8);
       byteBuffer.order(ByteOrder.LITTLE_ENDIAN);
       byteBuffer.putLong(number);
 
-      assertThat(buffer.toByteArrayUnsafe())
+      assertThat(buffer.unwrap())
         .containsExactly(byteBuffer.array());
     }
   }
@@ -190,10 +190,10 @@ public class BufferTest {
   @Test public void writeVarint_32() {
     int number = 300;
 
-    Buffer buffer = Buffer.allocate(Buffer.varintSizeInBytes(number));
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(number));
     buffer.writeVarint(number);
 
-    assertThat(buffer.toByteArrayUnsafe())
+    assertThat(buffer.unwrap())
       .containsExactly(0b1010_1100, 0b0000_0010);
   }
 
@@ -201,41 +201,41 @@ public class BufferTest {
   @Test public void writeVarint_64() {
     long number = 300;
 
-    Buffer buffer = Buffer.allocate(Buffer.varintSizeInBytes(number));
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(number));
     buffer.writeVarint(number);
 
-    assertThat(buffer.toByteArrayUnsafe())
+    assertThat(buffer.unwrap())
       .containsExactly(0b1010_1100, 0b0000_0010);
   }
 
   @Test public void writeVarint_ports() {
     // normal case
-    Buffer buffer = Buffer.allocate(Buffer.varintSizeInBytes(80));
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(80));
     buffer.writeVarint(80);
 
-    assertThat(buffer.toByteArrayUnsafe())
+    assertThat(buffer.unwrap())
       .containsExactly(0b0101_0000);
 
     // largest value to not require more than 2 bytes (14 bits set)
-    buffer = Buffer.allocate(Buffer.varintSizeInBytes(16383));
+    buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(16383));
     buffer.writeVarint(16383);
 
-    assertThat(buffer.toByteArrayUnsafe())
+    assertThat(buffer.unwrap())
       .containsExactly(0b1111_1111, 0b0111_1111);
 
     // worst case is a byte longer than fixed 16
-    buffer = Buffer.allocate(Buffer.varintSizeInBytes(65535));
+    buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(65535));
     buffer.writeVarint(65535);
 
-    assertThat(buffer.toByteArrayUnsafe())
+    assertThat(buffer.unwrap())
       .containsExactly(0b1111_1111, 0b1111_1111, 0b0000_0011);
 
     // most bits
-    buffer = Buffer.allocate(Buffer.varintSizeInBytes(0xFFFFFFFF));
+    buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(0xFFFFFFFF));
     buffer.writeVarint(0xFFFFFFFF);
 
     // we have a total of 32 bits encoded
-    assertThat(buffer.toByteArrayUnsafe())
+    assertThat(buffer.unwrap())
       .containsExactly(0b1111_1111, 0b1111_1111, 0b1111_1111, 0b1111_1111, 0b0000_1111);
   }
 
@@ -246,7 +246,7 @@ public class BufferTest {
   }
 
   static void assertReadVarint32(int value) {
-    Buffer buffer = Buffer.allocate(Buffer.varintSizeInBytes(value));
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(value));
     buffer.writeVarint(value);
     buffer.reset();
 
@@ -255,7 +255,7 @@ public class BufferTest {
   }
 
   @Test public void readVarint32_malformedTooBig() {
-    Buffer buffer = Buffer.allocate(8);
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(8);
     buffer.writeLongLe(0xffffffffffffL);
     buffer.reset();
 
@@ -275,7 +275,7 @@ public class BufferTest {
   }
 
   static void assertReadVarint64(long value) {
-    Buffer buffer = Buffer.allocate(Buffer.varintSizeInBytes(value));
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(value));
     buffer.writeVarint(value);
     buffer.reset();
 
@@ -284,7 +284,7 @@ public class BufferTest {
   }
 
   @Test public void readVarint64_malformedTooBig() {
-    Buffer buffer = Buffer.allocate(16);
+    UnsafeBuffer buffer = UnsafeBuffer.allocate(16);
     buffer.writeLongLe(0xffffffffffffffffL);
     buffer.writeLongLe(0xffffffffffffffffL);
     buffer.reset();
diff --git a/zipkin/src/test/java/zipkin2/internal/V1JsonSpanWriterTest.java b/zipkin/src/test/java/zipkin2/internal/V1JsonSpanWriterTest.java
index 9f0aa8a..7f6ae0a 100644
--- a/zipkin/src/test/java/zipkin2/internal/V1JsonSpanWriterTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/V1JsonSpanWriterTest.java
@@ -27,7 +27,7 @@ import static zipkin2.TestObjects.CLIENT_SPAN;
 
 public class V1JsonSpanWriterTest {
   V1JsonSpanWriter writer = new V1JsonSpanWriter();
-  Buffer buf = Buffer.allocate(2048); // bigger than needed to test sizeOf
+  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
 
   @Test
   public void sizeInBytes() {
@@ -65,7 +65,7 @@ public class V1JsonSpanWriterTest {
   }
 
   void writesCoreAnnotations(String begin, String end) throws UnsupportedEncodingException {
-    String json = new String(buf.toByteArrayUnsafe(), "UTF-8");
+    String json = new String(buf.unwrap(), "UTF-8");
 
     assertThat(json)
         .contains("{\"timestamp\":" + CLIENT_SPAN.timestamp() + ",\"value\":\"" + begin + "\"");
@@ -107,7 +107,7 @@ public class V1JsonSpanWriterTest {
   }
 
   void writesCoreSendAnnotations(String begin) throws UnsupportedEncodingException {
-    String json = new String(buf.toByteArrayUnsafe(), "UTF-8");
+    String json = new String(buf.unwrap(), "UTF-8");
 
     assertThat(json)
         .contains("{\"timestamp\":" + CLIENT_SPAN.timestamp() + ",\"value\":\"" + begin + "\"");
@@ -142,7 +142,7 @@ public class V1JsonSpanWriterTest {
   }
 
   void writesAddressBinaryAnnotation(String address) throws UnsupportedEncodingException {
-    String json = new String(buf.toByteArrayUnsafe(), "UTF-8");
+    String json = new String(buf.unwrap(), "UTF-8");
 
     assertThat(json).contains("{\"key\":\"" + address + "\",\"value\":true,\"endpoint\":");
   }
@@ -151,7 +151,7 @@ public class V1JsonSpanWriterTest {
   public void writes128BitTraceId() throws UnsupportedEncodingException {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
         .startsWith("{\"traceId\":\"" + CLIENT_SPAN.traceId() + "\"");
   }
 
@@ -159,7 +159,7 @@ public class V1JsonSpanWriterTest {
   public void annotationsHaveEndpoints() throws IOException {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
         .contains(
             "\"value\":\"foo\",\"endpoint\":{\"serviceName\":\"frontend\",\"ipv4\":\"127.0.0.1\"}");
   }
@@ -168,7 +168,7 @@ public class V1JsonSpanWriterTest {
   public void writesTimestampAndDuration() throws IOException {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
         .contains(
             "\"timestamp\":" + CLIENT_SPAN.timestamp() + ",\"duration\":" + CLIENT_SPAN.duration());
   }
@@ -177,7 +177,7 @@ public class V1JsonSpanWriterTest {
   public void skipsTimestampAndDuration_shared() throws IOException {
     writer.write(CLIENT_SPAN.toBuilder().kind(Span.Kind.SERVER).shared(true).build(), buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
         .doesNotContain(
             "\"timestamp\":" + CLIENT_SPAN.timestamp() + ",\"duration\":" + CLIENT_SPAN.duration());
   }
@@ -193,7 +193,7 @@ public class V1JsonSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8")).contains("\"name\":\"\"");
+    assertThat(new String(buf.unwrap(), "UTF-8")).contains("\"name\":\"\"");
   }
 
   @Test
@@ -206,7 +206,7 @@ public class V1JsonSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
         .contains("\"value\":\"foo\",\"endpoint\":{\"serviceName\":\"\",\"ipv4\":\"127.0.0.1\"}");
   }
 
@@ -214,7 +214,7 @@ public class V1JsonSpanWriterTest {
   public void tagsAreBinaryAnnotations() throws IOException {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
         .contains(
             "\"binaryAnnotations\":["
                 + "{\"key\":\"clnt/finagle.version\",\"value\":\"6.45.0\",\"endpoint\":{\"serviceName\":\"frontend\",\"ipv4\":\"127.0.0.1\"}},"
diff --git a/zipkin/src/test/java/zipkin2/internal/V1ThriftSpanWriterTest.java b/zipkin/src/test/java/zipkin2/internal/V1ThriftSpanWriterTest.java
index 54e0009..ee32150 100644
--- a/zipkin/src/test/java/zipkin2/internal/V1ThriftSpanWriterTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/V1ThriftSpanWriterTest.java
@@ -38,16 +38,16 @@ import static zipkin2.internal.ThriftField.TYPE_STRUCT;
 public class V1ThriftSpanWriterTest {
   Span span = Span.newBuilder().traceId("1").id("2").build();
   Endpoint endpoint = Endpoint.newBuilder().serviceName("frontend").ip("1.2.3.4").build();
-  Buffer buf = Buffer.allocate(2048); // bigger than needed to test sizeOf
+  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
 
   V1ThriftSpanWriter writer = new V1ThriftSpanWriter();
   byte[] endpointBytes;
 
   @Before
   public void init() {
-    Buffer endpointBuffer = Buffer.allocate(ThriftEndpointCodec.sizeInBytes(endpoint));
+    UnsafeBuffer endpointBuffer = UnsafeBuffer.allocate(ThriftEndpointCodec.sizeInBytes(endpoint));
     ThriftEndpointCodec.write(endpoint, endpointBuffer);
-    endpointBytes = endpointBuffer.toByteArrayUnsafe();
+    endpointBytes = endpointBuffer.unwrap();
   }
 
 
@@ -55,9 +55,9 @@ public class V1ThriftSpanWriterTest {
   public void endpoint_highPort() {
     int highPort = 63840;
     Endpoint endpoint = Endpoint.newBuilder().ip("127.0.0.1").port(63840).build();
-    Buffer endpointBuffer = Buffer.allocate(ThriftEndpointCodec.sizeInBytes(endpoint));
+    UnsafeBuffer endpointBuffer = UnsafeBuffer.allocate(ThriftEndpointCodec.sizeInBytes(endpoint));
     ThriftEndpointCodec.write(endpoint, endpointBuffer);
-    byte[] buff = endpointBuffer.toByteArrayUnsafe();
+    byte[] buff = endpointBuffer.unwrap();
 
     assertThat(buff)
       .containsSequence(TYPE_I32, 0, 1, 127, 0, 0, 1) // ipv4
@@ -103,9 +103,9 @@ public class V1ThriftSpanWriterTest {
 
   @Test
   public void writeList_offset_startsWithListPrefix() {
-    writer.writeList(asList(span, span), buf.toByteArrayUnsafe(), 1);
+    writer.writeList(asList(span, span), buf.unwrap(), 1);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .startsWith( // member type of the list and an integer with the count
             0, TYPE_STRUCT, 0, 0, 0, 2);
   }
@@ -114,10 +114,10 @@ public class V1ThriftSpanWriterTest {
   public void doesntWriteAnnotationsWhenMissingTimestamp() {
     writer.write(span.toBuilder().kind(CLIENT).build(), buf);
 
-    Buffer buf2 = Buffer.allocate(2048);
+    UnsafeBuffer buf2 = UnsafeBuffer.allocate(2048);
     writer.write(span, buf2);
 
-    assertThat(buf.toByteArrayUnsafe()).containsExactly(buf.toByteArrayUnsafe());
+    assertThat(buf.unwrap()).containsExactly(buf.unwrap());
   }
 
   @Test
@@ -144,7 +144,7 @@ public class V1ThriftSpanWriterTest {
     span = span.toBuilder().kind(kind).timestamp(5).duration(10).build();
     writer.write(span, buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(TYPE_LIST, 0, 6, TYPE_STRUCT, 0, 0, 0, 2) // two annotations
         .containsSequence(TYPE_I64, 0, 1, 0, 0, 0, 0, 0, 0, 0, 5) // timestamp
         .containsSequence(TYPE_STRING, 0, 2, 0, 0, 0, 2, begin.charAt(0), begin.charAt(1))
@@ -176,7 +176,7 @@ public class V1ThriftSpanWriterTest {
     span = span.toBuilder().kind(kind).timestamp(5).build();
     writer.write(span, buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(TYPE_LIST, 0, 6, TYPE_STRUCT, 0, 0, 0, 1) // one annotation
         .containsSequence(TYPE_I64, 0, 1, 0, 0, 0, 0, 0, 0, 0, 5) // timestamp
         .containsSequence(TYPE_STRING, 0, 2, 0, 0, 0, 2, begin.charAt(0), begin.charAt(1));
@@ -205,7 +205,7 @@ public class V1ThriftSpanWriterTest {
   void writesAddressBinaryAnnotation(Span.Kind kind, String addr) {
     writer.write(span.toBuilder().kind(kind).remoteEndpoint(endpoint).build(), buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(TYPE_LIST, 0, 8, TYPE_STRUCT, 0, 0, 0, 1) // one binary annotation
         .containsSequence(TYPE_STRING, 0, 1, 0, 0, 0, 2, addr.charAt(0), addr.charAt(1)) // key
         .containsSequence(TYPE_STRING, 0, 2, 0, 0, 0, 1, 1) // value
@@ -217,7 +217,7 @@ public class V1ThriftSpanWriterTest {
   public void annotationsHaveEndpoints() {
     writer.write(span.toBuilder().localEndpoint(endpoint).addAnnotation(5, "foo").build(), buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(TYPE_LIST, 0, 6, TYPE_STRUCT, 0, 0, 0, 1) // one annotation
         .containsSequence(TYPE_I64, 0, 1, 0, 0, 0, 0, 0, 0, 0, 5) // timestamp
         .containsSequence(TYPE_STRING, 0, 2, 0, 0, 0, 3, 'f', 'o', 'o') // value
@@ -228,7 +228,7 @@ public class V1ThriftSpanWriterTest {
   public void writesTimestampAndDuration() {
     writer.write(span.toBuilder().timestamp(5).duration(10).build(), buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(TYPE_I64, 0, 10, 0, 0, 0, 0, 0, 0, 0, 5) // timestamp
         .containsSequence(TYPE_I64, 0, 11, 0, 0, 0, 0, 0, 0, 0, 10); // duration
   }
@@ -237,10 +237,10 @@ public class V1ThriftSpanWriterTest {
   public void skipsTimestampAndDuration_shared() {
     writer.write(span.toBuilder().kind(SERVER).timestamp(5).duration(10).shared(true).build(), buf);
 
-    Buffer buf2 = Buffer.allocate(2048);
+    UnsafeBuffer buf2 = UnsafeBuffer.allocate(2048);
     writer.write(span.toBuilder().kind(SERVER).build(), buf2);
 
-    assertThat(buf.toByteArrayUnsafe()).containsExactly(buf.toByteArrayUnsafe());
+    assertThat(buf.unwrap()).containsExactly(buf.unwrap());
   }
 
   @Test
@@ -249,7 +249,7 @@ public class V1ThriftSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(
             ThriftField.TYPE_STRING, 0, 3, 0, 0, 0, 0); // name (empty is 32 zero bits)
   }
@@ -258,7 +258,7 @@ public class V1ThriftSpanWriterTest {
   public void writesTraceAndSpanIds() {
     writer.write(span, buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .startsWith(TYPE_I64, 0, 1, 0, 0, 0, 0, 0, 0, 0, 1) // trace ID
         .containsSequence(TYPE_I64, 0, 4, 0, 0, 0, 0, 0, 0, 0, 2); // ID
   }
@@ -269,7 +269,7 @@ public class V1ThriftSpanWriterTest {
         Span.newBuilder().traceId("00000000000000010000000000000002").parentId("3").id("4").build(),
         buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .startsWith(TYPE_I64, 0, 1, 0, 0, 0, 0, 0, 0, 0, 2) // trace ID
         .containsSequence(TYPE_I64, 0, 12, 0, 0, 0, 0, 0, 0, 0, 1) // trace ID high
         .containsSequence(TYPE_I64, 0, 5, 0, 0, 0, 0, 0, 0, 0, 3); // parent ID
@@ -282,7 +282,7 @@ public class V1ThriftSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(TYPE_LIST, 0, 6, TYPE_STRUCT, 0, 0, 0, 0) // empty annotations
         .containsSequence(TYPE_LIST, 0, 8, TYPE_STRUCT, 0, 0, 0, 0); // empty binary annotations
   }
@@ -293,7 +293,7 @@ public class V1ThriftSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(TYPE_LIST, 0, 8, TYPE_STRUCT, 0, 0, 0, 1) // one binary annotation
         .containsSequence(TYPE_STRING, 0, 1, 0, 0, 0, 2, 'l', 'c') // key
         .containsSequence(TYPE_STRING, 0, 2, 0, 0, 0, 0) // empty value
@@ -311,7 +311,7 @@ public class V1ThriftSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(
             ThriftField.TYPE_STRING, 0, 3, 0, 0, 0, 0); // serviceName (empty is 32 zero bits)
   }
@@ -323,14 +323,14 @@ public class V1ThriftSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(buf.toByteArrayUnsafe()).containsSequence(ThriftField.TYPE_BOOL, 0);
+    assertThat(buf.unwrap()).containsSequence(ThriftField.TYPE_BOOL, 0);
   }
 
   @Test
   public void tagsAreBinaryAnnotations() {
     writer.write(span.toBuilder().putTag("foo", "bar").build(), buf);
 
-    assertThat(buf.toByteArrayUnsafe())
+    assertThat(buf.unwrap())
         .containsSequence(TYPE_LIST, 0, 8, TYPE_STRUCT, 0, 0, 0, 1) // one binary annotation
         .containsSequence(TYPE_STRING, 0, 1, 0, 0, 0, 3, 'f', 'o', 'o') // key
         .containsSequence(TYPE_STRING, 0, 2, 0, 0, 0, 3, 'b', 'a', 'r') // value
diff --git a/zipkin/src/test/java/zipkin2/internal/V2SpanWriterTest.java b/zipkin/src/test/java/zipkin2/internal/V2SpanWriterTest.java
index 801a483..8c632ed 100644
--- a/zipkin/src/test/java/zipkin2/internal/V2SpanWriterTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/V2SpanWriterTest.java
@@ -30,7 +30,7 @@ import static zipkin2.TestObjects.TODAY;
 
 public class V2SpanWriterTest {
   V2SpanWriter writer = new V2SpanWriter();
-  Buffer buf = Buffer.allocate(2048); // bigger than needed to test sizeOf
+  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
 
   @Rule public ExpectedException thrown = ExpectedException.none();
 
@@ -43,14 +43,14 @@ public class V2SpanWriterTest {
   @Test public void writes128BitTraceId() throws UnsupportedEncodingException {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
       .startsWith("{\"traceId\":\"" + CLIENT_SPAN.traceId() + "\"");
   }
 
   @Test public void writesAnnotationWithoutEndpoint() throws IOException {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
       .contains("{\"timestamp\":" + (TODAY + 100) * 1000L + ",\"value\":\"foo\"}");
   }
 
@@ -63,7 +63,7 @@ public class V2SpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
       .doesNotContain("name");
   }
 
@@ -74,14 +74,14 @@ public class V2SpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
       .contains("\"localEndpoint\":{\"ipv4\":\"127.0.0.1\"}");
   }
 
   @Test public void tagsAreAMap() throws IOException {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.toByteArrayUnsafe(), "UTF-8"))
+    assertThat(new String(buf.unwrap(), "UTF-8"))
       .contains("\"tags\":{\"clnt/finagle.version\":\"6.45.0\",\"http.path\":\"/api\"}");
   }
 }