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/12 11:03:57 UTC

[incubator-zipkin] 01/01: Makes SpanBytesDecoder work on ByteBuffer

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

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

commit 5cdfb3d8bed682ce5aec39c6ee6fe940615460b4
Author: Adrian Cole <ac...@pivotal.io>
AuthorDate: Sun May 12 19:00:07 2019 +0800

    Makes SpanBytesDecoder work on ByteBuffer
    
    Internally, this also covers InputStream.
    
    This was easier (though not easy) due to splitting how we handle
    encoding from decoding.
    
    Incidentally, the jar size of zipkin is smaller still vs the last
    release.
    
    Fixes #2435
---
 .../java/zipkin2/codec/ProtoCodecBenchmarks.java   |   2 +-
 .../zipkin2/internal/UnsafeBufferBenchmarks.java   | 103 -----
 .../zipkin2/internal/Proto3CodecInteropTest.java   |  34 +-
 .../zipkin2/server/internal/ZipkinQueryApiV2.java  |   8 +-
 .../storage/cassandra/v1/CassandraUtil.java        |   1 -
 .../storage/cassandra/v1/SelectFromTraces.java     |  25 +-
 zipkin/src/main/java/zipkin2/Endpoint.java         |   2 +-
 zipkin/src/main/java/zipkin2/Span.java             |   2 +-
 .../zipkin2/codec/DependencyLinkBytesDecoder.java  |  75 ++--
 .../zipkin2/codec/DependencyLinkBytesEncoder.java  |   9 +-
 .../main/java/zipkin2/codec/SpanBytesDecoder.java  | 167 +++++---
 .../main/java/zipkin2/internal/Dependencies.java   |  47 ++-
 .../src/main/java/zipkin2/internal/HexCodec.java   |   4 +
 .../src/main/java/zipkin2/internal/JsonCodec.java  |  69 ++--
 .../main/java/zipkin2/internal/JsonEscaper.java    |   2 +-
 .../main/java/zipkin2/internal/Proto3Codec.java    |  14 +-
 .../main/java/zipkin2/internal/Proto3Fields.java   |  53 +--
 .../java/zipkin2/internal/Proto3SpanWriter.java    |  21 +-
 .../java/zipkin2/internal/Proto3ZipkinFields.java  |  24 +-
 .../main/java/zipkin2/internal/ThriftCodec.java    | 136 ++----
 .../java/zipkin2/internal/ThriftEndpointCodec.java |  22 +-
 .../main/java/zipkin2/internal/ThriftField.java    |  10 +-
 .../main/java/zipkin2/internal/UnsafeBuffer.java   | 454 ---------------------
 .../java/zipkin2/internal/V1JsonSpanReader.java    |  12 +-
 .../java/zipkin2/internal/V1JsonSpanWriter.java    |   8 +-
 .../main/java/zipkin2/internal/V1SpanWriter.java   |  22 +-
 .../java/zipkin2/internal/V1ThriftSpanReader.java  |  73 ++--
 .../java/zipkin2/internal/V1ThriftSpanWriter.java  |  42 +-
 .../main/java/zipkin2/internal/V2SpanWriter.java   |  18 +-
 .../java/zipkin2/codec/SpanBytesDecoderTest.java   |   6 +-
 .../java/zipkin2/codec/V1SpanBytesDecoderTest.java |   7 +-
 .../java/zipkin2/internal/DependenciesTest.java    |   5 +-
 .../test/java/zipkin2/internal/JsonCodecTest.java  |  10 +-
 .../java/zipkin2/internal/Proto3FieldsTest.java    |  42 +-
 .../zipkin2/internal/Proto3SpanWriterTest.java     |  19 +-
 .../zipkin2/internal/Proto3ZipkinFieldsTest.java   |  26 +-
 .../java/zipkin2/internal/UnsafeBufferTest.java    | 300 --------------
 .../zipkin2/internal/V1JsonSpanWriterTest.java     |  82 ++--
 .../zipkin2/internal/V1ThriftSpanWriterTest.java   | 177 ++++----
 .../java/zipkin2/internal/V2SpanWriterTest.java    |  26 +-
 40 files changed, 618 insertions(+), 1541 deletions(-)

diff --git a/benchmarks/src/main/java/zipkin2/codec/ProtoCodecBenchmarks.java b/benchmarks/src/main/java/zipkin2/codec/ProtoCodecBenchmarks.java
index a863158..36beb43 100644
--- a/benchmarks/src/main/java/zipkin2/codec/ProtoCodecBenchmarks.java
+++ b/benchmarks/src/main/java/zipkin2/codec/ProtoCodecBenchmarks.java
@@ -87,7 +87,7 @@ public class ProtoCodecBenchmarks {
 
   @Benchmark
   public List<Span> bytebuffer_zipkinDecoder() {
-    return SpanBytesDecoder.PROTO3.decodeList(ByteBufUtil.getBytes(encodedBuf));
+    return SpanBytesDecoder.PROTO3.decodeList(encodedBuf.nioBuffer());
   }
 
   @Benchmark
diff --git a/benchmarks/src/main/java/zipkin2/internal/UnsafeBufferBenchmarks.java b/benchmarks/src/main/java/zipkin2/internal/UnsafeBufferBenchmarks.java
deleted file mode 100644
index dac8060..0000000
--- a/benchmarks/src/main/java/zipkin2/internal/UnsafeBufferBenchmarks.java
+++ /dev/null
@@ -1,103 +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;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.concurrent.TimeUnit;
-import org.openjdk.jmh.annotations.Benchmark;
-import org.openjdk.jmh.annotations.BenchmarkMode;
-import org.openjdk.jmh.annotations.Fork;
-import org.openjdk.jmh.annotations.Measurement;
-import org.openjdk.jmh.annotations.Mode;
-import org.openjdk.jmh.annotations.OutputTimeUnit;
-import org.openjdk.jmh.annotations.Scope;
-import org.openjdk.jmh.annotations.State;
-import org.openjdk.jmh.annotations.Threads;
-import org.openjdk.jmh.annotations.Warmup;
-import org.openjdk.jmh.runner.Runner;
-import org.openjdk.jmh.runner.RunnerException;
-import org.openjdk.jmh.runner.options.Options;
-import org.openjdk.jmh.runner.options.OptionsBuilder;
-
-@Measurement(iterations = 5, time = 1)
-@Warmup(iterations = 10, time = 1)
-@Fork(3)
-@BenchmarkMode(Mode.AverageTime)
-@OutputTimeUnit(TimeUnit.MICROSECONDS)
-@State(Scope.Thread)
-@Threads(1)
-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不存在";
-  static final int CHINESE_UTF8_SIZE = UTF_8.encode(CHINESE_UTF8).remaining();
-  /* length-prefixing a 1 KiB span */
-  static final int TEST_INT = 1024;
-  /* epoch micros timestamp */
-  static final long TEST_LONG = 1472470996199000L;
-  UnsafeBuffer buffer = UnsafeBuffer.allocate(8);
-
-  @Benchmark public int utf8SizeInBytes_chinese() {
-    return UnsafeBuffer.utf8SizeInBytes(CHINESE_UTF8);
-  }
-
-  @Benchmark public byte[] writeUtf8_chinese() {
-    UnsafeBuffer bufferUtf8 = UnsafeBuffer.allocate(CHINESE_UTF8_SIZE);
-    bufferUtf8.writeUtf8(CHINESE_UTF8);
-    return bufferUtf8.unwrap();
-  }
-
-  @Benchmark public ByteBuffer writeUtf8_chinese_jdk() {
-    return UTF_8.encode(CHINESE_UTF8);
-  }
-
-  @Benchmark public int varIntSizeInBytes_32() {
-    return UnsafeBuffer.varintSizeInBytes(TEST_INT);
-  }
-
-  @Benchmark public int varIntSizeInBytes_64() {
-    return UnsafeBuffer.varintSizeInBytes(TEST_LONG);
-  }
-
-  @Benchmark public int writeVarint_32() {
-    buffer.reset();
-    buffer.writeVarint(TEST_INT);
-    return buffer.pos();
-  }
-
-  @Benchmark public int writeVarint_64() {
-    buffer.reset();
-    buffer.writeVarint(TEST_LONG);
-    return buffer.pos();
-  }
-
-  @Benchmark public int writeLongLe() {
-    buffer.reset();
-    buffer.writeLongLe(TEST_LONG);
-    return buffer.pos();
-  }
-
-  // Convenience main entry-point
-  public static void main(String[] args) throws RunnerException {
-    Options opt = new OptionsBuilder()
-      .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 d475538..351b0d3 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();
 
-    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(ANNOTATION.sizeInBytes(zipkinAnnotation));
-    ANNOTATION.write(zipkinBytes, zipkinAnnotation);
+    byte[] zipkinBytes = new byte[ANNOTATION.sizeInBytes(zipkinAnnotation)];
+    ANNOTATION.write(WriteBuffer.wrap(zipkinBytes, 0), zipkinAnnotation);
 
-    assertThat(zipkinBytes.unwrap())
+    assertThat(zipkinBytes)
       .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();
 
-    UnsafeBuffer wireBytes = UnsafeBuffer.wrap(wireSpan.encode(), 0);
+    ReadBuffer wireBytes = ReadBuffer.wrap(wireSpan.encode(), 0);
     assertThat(wireBytes.readVarint32())
       .isEqualTo(ANNOTATION.key);
 
@@ -179,7 +179,7 @@ public class Proto3CodecInteropTest {
 
   @Test public void endpoint_sizeInBytes_matchesWireEncodingWithTag() {
     assertThat(LOCAL_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.localEndpoint())).isEqualTo(
-        Endpoint.ADAPTER.encodedSizeWithTag(LOCAL_ENDPOINT.fieldNumber, PROTO_SPAN.local_endpoint)
+      Endpoint.ADAPTER.encodedSizeWithTag(LOCAL_ENDPOINT.fieldNumber, PROTO_SPAN.local_endpoint)
     );
 
     assertThat(REMOTE_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.remoteEndpoint())).isEqualTo(
@@ -188,20 +188,20 @@ public class Proto3CodecInteropTest {
   }
 
   @Test public void localEndpoint_write_matchesWire() {
-    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(LOCAL_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.localEndpoint()));
-    LOCAL_ENDPOINT.write(zipkinBytes, ZIPKIN_SPAN.localEndpoint());
+    byte[] zipkinBytes = new byte[LOCAL_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.localEndpoint())];
+    LOCAL_ENDPOINT.write(WriteBuffer.wrap(zipkinBytes, 0), ZIPKIN_SPAN.localEndpoint());
     Span wireSpan = new Span.Builder().local_endpoint(PROTO_SPAN.local_endpoint).build();
 
-    assertThat(zipkinBytes.unwrap())
+    assertThat(zipkinBytes)
       .containsExactly(wireSpan.encode());
   }
 
   @Test public void remoteEndpoint_write_matchesWire() {
-    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(REMOTE_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.remoteEndpoint()));
-    REMOTE_ENDPOINT.write(zipkinBytes, ZIPKIN_SPAN.remoteEndpoint());
+    byte[] zipkinBytes = new byte[REMOTE_ENDPOINT.sizeInBytes(ZIPKIN_SPAN.remoteEndpoint())];
+    REMOTE_ENDPOINT.write(WriteBuffer.wrap(zipkinBytes, 0), ZIPKIN_SPAN.remoteEndpoint());
     Span wireSpan = new Span.Builder().remote_endpoint(PROTO_SPAN.remote_endpoint).build();
 
-    assertThat(zipkinBytes.unwrap())
+    assertThat(zipkinBytes)
       .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);
-    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(field.sizeInBytes(entry));
-    field.write(zipkinBytes, entry);
+    byte[] zipkinBytes = new byte[field.sizeInBytes(entry)];
+    field.write(WriteBuffer.wrap(zipkinBytes, 0), entry);
 
     Span oneField = new Span.Builder().tags(singletonMap(entry.key, entry.value)).build();
-    assertThat(zipkinBytes.unwrap())
+    assertThat(zipkinBytes)
       .containsExactly(oneField.encode());
   }
 
   @Test public void writeTagField_matchesWire_emptyValue() {
     MapEntry<String, String> entry = entry("error", "");
     TagField field = new TagField(TAG_KEY);
-    UnsafeBuffer zipkinBytes = UnsafeBuffer.allocate(field.sizeInBytes(entry));
-    field.write(zipkinBytes, entry);
+    byte[] zipkinBytes = new byte[field.sizeInBytes(entry)];
+    field.write(WriteBuffer.wrap(zipkinBytes, 0), entry);
 
     Span oneField = new Span.Builder().tags(singletonMap(entry.key, entry.value)).build();
-    assertThat(zipkinBytes.unwrap())
+    assertThat(zipkinBytes)
       .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 265a89b..4427f79 100644
--- a/zipkin-server/src/main/java/zipkin2/server/internal/ZipkinQueryApiV2.java
+++ b/zipkin-server/src/main/java/zipkin2/server/internal/ZipkinQueryApiV2.java
@@ -40,7 +40,7 @@ import zipkin2.Span;
 import zipkin2.codec.DependencyLinkBytesEncoder;
 import zipkin2.codec.SpanBytesEncoder;
 import zipkin2.internal.JsonCodec;
-import zipkin2.internal.UnsafeBuffer;
+import zipkin2.internal.WriteBuffer;
 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 UnsafeBuffer.Writer<String> QUOTED_STRING_WRITER = new UnsafeBuffer.Writer<String>() {
+  static final WriteBuffer.Writer<String> QUOTED_STRING_WRITER = new WriteBuffer.Writer<String>() {
     @Override public int sizeInBytes(String value) {
-      return UnsafeBuffer.utf8SizeInBytes(value) + 2; // quotes
+      return WriteBuffer.utf8SizeInBytes(value) + 2; // quotes
     }
 
-    @Override public void write(String value, UnsafeBuffer buffer) {
+    @Override public void write(String value, WriteBuffer buffer) {
       buffer.writeByte('"');
       buffer.writeUtf8(value);
       buffer.writeByte('"');
diff --git a/zipkin-storage/cassandra-v1/src/main/java/zipkin2/storage/cassandra/v1/CassandraUtil.java b/zipkin-storage/cassandra-v1/src/main/java/zipkin2/storage/cassandra/v1/CassandraUtil.java
index 4510e71..39e268c 100644
--- a/zipkin-storage/cassandra-v1/src/main/java/zipkin2/storage/cassandra/v1/CassandraUtil.java
+++ b/zipkin-storage/cassandra-v1/src/main/java/zipkin2/storage/cassandra/v1/CassandraUtil.java
@@ -38,7 +38,6 @@ import zipkin2.Call;
 import zipkin2.Span;
 import zipkin2.internal.Nullable;
 import zipkin2.internal.Platform;
-import zipkin2.internal.UnsafeBuffer;
 import zipkin2.storage.QueryRequest;
 
 import static com.google.common.base.Preconditions.checkArgument;
diff --git a/zipkin-storage/cassandra-v1/src/main/java/zipkin2/storage/cassandra/v1/SelectFromTraces.java b/zipkin-storage/cassandra-v1/src/main/java/zipkin2/storage/cassandra/v1/SelectFromTraces.java
index f9636f2..1064578 100644
--- a/zipkin-storage/cassandra-v1/src/main/java/zipkin2/storage/cassandra/v1/SelectFromTraces.java
+++ b/zipkin-storage/cassandra-v1/src/main/java/zipkin2/storage/cassandra/v1/SelectFromTraces.java
@@ -35,6 +35,7 @@ import zipkin2.Span;
 import zipkin2.internal.FilterTraces;
 import zipkin2.internal.HexCodec;
 import zipkin2.internal.Nullable;
+import zipkin2.internal.ReadBuffer;
 import zipkin2.internal.V1ThriftSpanReader;
 import zipkin2.storage.GroupByTraceId;
 import zipkin2.storage.QueryRequest;
@@ -59,11 +60,11 @@ final class SelectFromTraces extends ResultSetFutureCall<ResultSet> {
       this.accumulateSpans = new DecodeAndConvertSpans();
 
       this.preparedStatement =
-          session.prepare(
-              QueryBuilder.select("trace_id", "span")
-                  .from("traces")
-                  .where(QueryBuilder.in("trace_id", QueryBuilder.bindMarker("trace_id")))
-                  .limit(QueryBuilder.bindMarker("limit_")));
+        session.prepare(
+          QueryBuilder.select("trace_id", "span")
+            .from("traces")
+            .where(QueryBuilder.in("trace_id", QueryBuilder.bindMarker("trace_id")))
+            .limit(QueryBuilder.bindMarker("limit_")));
       this.maxTraceCols = maxTraceCols;
       this.strictTraceId = strictTraceId;
       this.groupByTraceId = GroupByTraceId.create(strictTraceId);
@@ -72,8 +73,8 @@ final class SelectFromTraces extends ResultSetFutureCall<ResultSet> {
     Call<List<Span>> newCall(String hexTraceId) {
       long traceId = HexCodec.lowerHexToUnsignedLong(hexTraceId);
       Call<List<Span>> result =
-          new SelectFromTraces(this, Collections.singleton(traceId), maxTraceCols)
-              .flatMap(accumulateSpans);
+        new SelectFromTraces(this, Collections.singleton(traceId), maxTraceCols)
+          .flatMap(accumulateSpans);
       return strictTraceId ? result.map(StrictTraceId.filterSpans(hexTraceId)) : result;
     }
 
@@ -95,7 +96,7 @@ final class SelectFromTraces extends ResultSetFutureCall<ResultSet> {
   @Override
   protected ResultSetFuture newFuture() {
     return factory.session.executeAsync(
-        factory.preparedStatement.bind().setSet("trace_id", trace_id).setInt("limit_", limit_));
+      factory.preparedStatement.bind().setSet("trace_id", trace_id).setInt("limit_", limit_));
   }
 
   @Override public ResultSet map(ResultSet input) {
@@ -139,9 +140,9 @@ final class SelectFromTraces extends ResultSetFutureCall<ResultSet> {
         traceIds = input;
       }
       Call<List<List<Span>>> result =
-          new SelectFromTraces(factory, traceIds, factory.maxTraceCols)
-              .flatMap(factory.accumulateSpans)
-              .map(factory.groupByTraceId);
+        new SelectFromTraces(factory, traceIds, factory.maxTraceCols)
+          .flatMap(factory.accumulateSpans)
+          .map(factory.groupByTraceId);
       return filter != null ? result.map(filter) : result;
     }
 
@@ -163,7 +164,7 @@ final class SelectFromTraces extends ResultSetFutureCall<ResultSet> {
       return (row, result) -> {
         V1ThriftSpanReader reader = V1ThriftSpanReader.create();
         V1SpanConverter converter = V1SpanConverter.create();
-        V1Span read = reader.read(row.getBytes("span"));
+        V1Span read = reader.read(ReadBuffer.wrapUnsafe(row.getBytes("span")));
         converter.convert(read, result);
       };
     }
diff --git a/zipkin/src/main/java/zipkin2/Endpoint.java b/zipkin/src/main/java/zipkin2/Endpoint.java
index 9684541..2b7ddbe 100644
--- a/zipkin/src/main/java/zipkin2/Endpoint.java
+++ b/zipkin/src/main/java/zipkin2/Endpoint.java
@@ -27,7 +27,7 @@ import java.util.Locale;
 import zipkin2.internal.Nullable;
 import zipkin2.internal.Platform;
 
-import static zipkin2.internal.UnsafeBuffer.HEX_DIGITS;
+import static zipkin2.internal.HexCodec.HEX_DIGITS;
 
 /** The network context of a node in the service graph. */
 //@Immutable
diff --git a/zipkin/src/main/java/zipkin2/Span.java b/zipkin/src/main/java/zipkin2/Span.java
index 6ea685d..e6b541d 100644
--- a/zipkin/src/main/java/zipkin2/Span.java
+++ b/zipkin/src/main/java/zipkin2/Span.java
@@ -36,7 +36,7 @@ import zipkin2.internal.Platform;
 
 import static java.lang.String.format;
 import static java.util.logging.Level.FINEST;
-import static zipkin2.internal.UnsafeBuffer.HEX_DIGITS;
+import static zipkin2.internal.HexCodec.HEX_DIGITS;
 
 /**
  * A span is a single-host view of an operation. A trace is a series of spans (often RPC calls)
diff --git a/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesDecoder.java b/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesDecoder.java
index 90334c0..66c0ff4 100644
--- a/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesDecoder.java
+++ b/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesDecoder.java
@@ -24,66 +24,59 @@ import java.util.List;
 import zipkin2.DependencyLink;
 import zipkin2.internal.JsonCodec;
 import zipkin2.internal.JsonCodec.JsonReader;
+import zipkin2.internal.JsonCodec.JsonReaderAdapter;
 import zipkin2.internal.Nullable;
+import zipkin2.internal.ReadBuffer;
 
 public enum DependencyLinkBytesDecoder implements BytesDecoder<DependencyLink> {
   JSON_V1 {
-    @Override
-    public Encoding encoding() {
+    @Override public Encoding encoding() {
       return Encoding.JSON;
     }
 
-    @Override
-    public boolean decode(byte[] link, Collection<DependencyLink> out) {
-      return JsonCodec.read(READER, link, out);
+    @Override public boolean decode(byte[] link, Collection<DependencyLink> out) {
+      return JsonCodec.read(READER, ReadBuffer.wrap(link, 0), out);
     }
 
-    @Override
-    @Nullable
-    public DependencyLink decodeOne(byte[] link) {
-      return JsonCodec.readOne(READER, link);
+    @Override @Nullable public DependencyLink decodeOne(byte[] link) {
+      return JsonCodec.readOne(READER, ReadBuffer.wrap(link, 0));
     }
 
-    @Override
-    public boolean decodeList(byte[] links, Collection<DependencyLink> out) {
-      return JsonCodec.readList(READER, links, out);
+    @Override public boolean decodeList(byte[] links, Collection<DependencyLink> out) {
+      return JsonCodec.readList(READER, ReadBuffer.wrap(links, 0), out);
     }
 
-    @Override
-    public List<DependencyLink> decodeList(byte[] links) {
+    @Override public List<DependencyLink> decodeList(byte[] links) {
       List<DependencyLink> out = new ArrayList<>();
       if (!decodeList(links, out)) return Collections.emptyList();
       return out;
     }
   };
 
-  static final JsonCodec.JsonReaderAdapter<DependencyLink> READER =
-      new JsonCodec.JsonReaderAdapter<DependencyLink>() {
-        @Override
-        public DependencyLink fromJson(JsonReader reader) throws IOException {
-          DependencyLink.Builder result = DependencyLink.newBuilder();
-          reader.beginObject();
-          while (reader.hasNext()) {
-            String nextName = reader.nextName();
-            if (nextName.equals("parent")) {
-              result.parent(reader.nextString());
-            } else if (nextName.equals("child")) {
-              result.child(reader.nextString());
-            } else if (nextName.equals("callCount")) {
-              result.callCount(reader.nextLong());
-            } else if (nextName.equals("errorCount")) {
-              result.errorCount(reader.nextLong());
-            } else {
-              reader.skipValue();
-            }
-          }
-          reader.endObject();
-          return result.build();
+  static final JsonReaderAdapter<DependencyLink> READER = new JsonReaderAdapter<DependencyLink>() {
+    @Override public DependencyLink fromJson(JsonReader reader) throws IOException {
+      DependencyLink.Builder result = DependencyLink.newBuilder();
+      reader.beginObject();
+      while (reader.hasNext()) {
+        String nextName = reader.nextName();
+        if (nextName.equals("parent")) {
+          result.parent(reader.nextString());
+        } else if (nextName.equals("child")) {
+          result.child(reader.nextString());
+        } else if (nextName.equals("callCount")) {
+          result.callCount(reader.nextLong());
+        } else if (nextName.equals("errorCount")) {
+          result.errorCount(reader.nextLong());
+        } else {
+          reader.skipValue();
         }
+      }
+      reader.endObject();
+      return result.build();
+    }
 
-        @Override
-        public String toString() {
-          return "DependencyLink";
-        }
-      };
+    @Override public String toString() {
+      return "DependencyLink";
+    }
+  };
 }
diff --git a/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java b/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java
index aee13b5..12973e4 100644
--- a/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java
+++ b/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java
@@ -19,11 +19,12 @@ package zipkin2.codec;
 import java.util.List;
 import zipkin2.DependencyLink;
 import zipkin2.internal.JsonCodec;
-import zipkin2.internal.UnsafeBuffer;
+import zipkin2.internal.WriteBuffer;
+import zipkin2.internal.WriteBuffer.Writer;
 
 import static zipkin2.internal.JsonEscaper.jsonEscape;
 import static zipkin2.internal.JsonEscaper.jsonEscapedSizeInBytes;
-import static zipkin2.internal.UnsafeBuffer.asciiSizeInBytes;
+import static zipkin2.internal.WriteBuffer.asciiSizeInBytes;
 
 public enum DependencyLinkBytesEncoder implements BytesEncoder<DependencyLink> {
   JSON_V1 {
@@ -44,7 +45,7 @@ public enum DependencyLinkBytesEncoder implements BytesEncoder<DependencyLink> {
     }
   };
 
-  static final UnsafeBuffer.Writer<DependencyLink> WRITER = new UnsafeBuffer.Writer<DependencyLink>() {
+  static final Writer<DependencyLink> WRITER = new Writer<DependencyLink>() {
     @Override public int sizeInBytes(DependencyLink value) {
       int sizeInBytes = 37; // {"parent":"","child":"","callCount":}
       sizeInBytes += jsonEscapedSizeInBytes(value.parent());
@@ -57,7 +58,7 @@ public enum DependencyLinkBytesEncoder implements BytesEncoder<DependencyLink> {
       return sizeInBytes;
     }
 
-    @Override public void write(DependencyLink value, UnsafeBuffer b) {
+    @Override public void write(DependencyLink value, WriteBuffer b) {
       b.writeAscii("{\"parent\":\"");
       b.writeUtf8(jsonEscape(value.parent()));
       b.writeAscii("\",\"child\":\"");
diff --git a/zipkin/src/main/java/zipkin2/codec/SpanBytesDecoder.java b/zipkin/src/main/java/zipkin2/codec/SpanBytesDecoder.java
index 7c3e290..96218ee 100644
--- a/zipkin/src/main/java/zipkin2/codec/SpanBytesDecoder.java
+++ b/zipkin/src/main/java/zipkin2/codec/SpanBytesDecoder.java
@@ -16,6 +16,7 @@
  */
 package zipkin2.codec;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -24,6 +25,7 @@ import zipkin2.Span;
 import zipkin2.internal.JsonCodec;
 import zipkin2.internal.Nullable;
 import zipkin2.internal.Proto3Codec;
+import zipkin2.internal.ReadBuffer;
 import zipkin2.internal.ThriftCodec;
 import zipkin2.internal.V1JsonSpanReader;
 import zipkin2.internal.V2SpanReader;
@@ -35,123 +37,180 @@ import zipkin2.v1.V1SpanConverter;
 public enum SpanBytesDecoder implements BytesDecoder<Span> {
   /** Corresponds to the Zipkin v1 json format */
   JSON_V1 {
-    @Override
-    public Encoding encoding() {
+    @Override public Encoding encoding() {
       return Encoding.JSON;
     }
 
-    @Override
-    public boolean decode(byte[] bytes, Collection<Span> out) {
-      Span result = decodeOne(bytes);
+    @Override public boolean decode(byte[] span, Collection<Span> out) { // ex DependencyLinker
+      Span result = decodeOne(ReadBuffer.wrap(span, 0));
       if (result == null) return false;
       out.add(result);
       return true;
     }
 
-    @Override
-    public boolean decodeList(byte[] spans, Collection<Span> out) {
-      return new V1JsonSpanReader().readList(spans, out);
+    @Override public boolean decodeList(byte[] spans, Collection<Span> out) { // ex getTrace
+      return new V1JsonSpanReader().readList(ReadBuffer.wrap(spans, 0), out);
     }
 
-    @Override
-    public Span decodeOne(byte[] span) {
-      V1Span v1 = JsonCodec.readOne(new V1JsonSpanReader(), span);
+    @Override public boolean decodeList(ByteBuffer spans, Collection<Span> out) {
+      return new V1JsonSpanReader().readList(ReadBuffer.wrapUnsafe(spans), out);
+    }
+
+    @Override @Nullable public Span decodeOne(byte[] span) {
+      return decodeOne(ReadBuffer.wrap(span, 0));
+    }
+
+    @Override @Nullable public Span decodeOne(ByteBuffer span) {
+      return decodeOne(ReadBuffer.wrapUnsafe(span));
+    }
+
+    Span decodeOne(ReadBuffer buffer) {
+      V1Span v1 = JsonCodec.readOne(new V1JsonSpanReader(), buffer);
       List<Span> out = new ArrayList<>(1);
       V1SpanConverter.create().convert(v1, out);
       return out.get(0);
     }
 
-    @Override
-    public List<Span> decodeList(byte[] spans) {
+    @Override public List<Span> decodeList(byte[] spans) {
+      return decodeList(this, spans);
+    }
+
+    @Override public List<Span> decodeList(ByteBuffer spans) {
       return decodeList(this, spans);
     }
   },
   /** Corresponds to the Zipkin v1 thrift format */
   THRIFT {
-    @Override
-    public Encoding encoding() {
+    @Override public Encoding encoding() {
       return Encoding.THRIFT;
     }
 
-    @Override
-    public boolean decode(byte[] span, Collection<Span> out) {
-      return ThriftCodec.read(span, out);
+    @Override public boolean decode(byte[] span, Collection<Span> out) { // ex DependencyLinker
+      return ThriftCodec.read(ReadBuffer.wrap(span, 0), out);
     }
 
-    @Override
-    public boolean decodeList(byte[] spans, Collection<Span> out) {
-      return ThriftCodec.readList(spans, out);
+    @Override public boolean decodeList(byte[] spans, Collection<Span> out) { // ex getTrace
+      return ThriftCodec.readList(ReadBuffer.wrap(spans, 0), out);
     }
 
-    @Override
-    public Span decodeOne(byte[] span) {
-      return ThriftCodec.readOne(span);
+    @Override public boolean decodeList(ByteBuffer spans, Collection<Span> out) {
+      return ThriftCodec.readList(ReadBuffer.wrapUnsafe(spans), out);
     }
 
-    @Override
-    public List<Span> decodeList(byte[] spans) {
+    @Override @Nullable public Span decodeOne(byte[] span) {
+      return ThriftCodec.readOne(ReadBuffer.wrap(span, 0));
+    }
+
+    @Override @Nullable public Span decodeOne(ByteBuffer span) {
+      return ThriftCodec.readOne(ReadBuffer.wrapUnsafe(span));
+    }
+
+    @Override public List<Span> decodeList(byte[] spans) {
+      return decodeList(this, spans);
+    }
+
+    @Override public List<Span> decodeList(ByteBuffer spans) {
       return decodeList(this, spans);
     }
   },
   /** Corresponds to the Zipkin v2 json format */
   JSON_V2 {
-    @Override
-    public Encoding encoding() {
+    @Override public Encoding encoding() {
       return Encoding.JSON;
     }
 
-    @Override
-    public boolean decode(byte[] span, Collection<Span> out) { // ex decode span in dependencies job
-      return JsonCodec.read(new V2SpanReader(), span, out);
+    @Override public boolean decode(byte[] span, Collection<Span> out) { // ex DependencyLinker
+      return JsonCodec.read(new V2SpanReader(), ReadBuffer.wrap(span, 0), out);
+    }
+
+    @Override public boolean decodeList(byte[] spans, Collection<Span> out) { // ex getTrace
+      return JsonCodec.readList(new V2SpanReader(), ReadBuffer.wrap(spans, 0), out);
     }
 
-    @Override
-    public boolean decodeList(byte[] spans, Collection<Span> out) { // ex getTrace
-      return JsonCodec.readList(new V2SpanReader(), spans, out);
+    @Override public boolean decodeList(ByteBuffer spans, Collection<Span> out) {
+      return JsonCodec.readList(new V2SpanReader(), ReadBuffer.wrapUnsafe(spans), out);
     }
 
-    @Override
-    @Nullable
-    public Span decodeOne(byte[] span) {
-      return JsonCodec.readOne(new V2SpanReader(), span);
+    @Override @Nullable public Span decodeOne(byte[] span) {
+      return JsonCodec.readOne(new V2SpanReader(), ReadBuffer.wrap(span, 0));
     }
 
-    @Override
-    public List<Span> decodeList(byte[] spans) {
+    @Override @Nullable public Span decodeOne(ByteBuffer span) {
+      return JsonCodec.readOne(new V2SpanReader(), ReadBuffer.wrapUnsafe(span));
+    }
+
+    @Override public List<Span> decodeList(byte[] spans) {
+      return decodeList(this, spans);
+    }
+
+    @Override public List<Span> decodeList(ByteBuffer spans) {
       return decodeList(this, spans);
     }
   },
   PROTO3 {
-    @Override
-    public Encoding encoding() {
+    @Override public Encoding encoding() {
       return Encoding.PROTO3;
     }
 
-    @Override
-    public boolean decode(byte[] span, Collection<Span> out) { // ex decode span in dependencies job
-      return Proto3Codec.read(span, out);
+    @Override public boolean decode(byte[] span, Collection<Span> out) { // ex DependencyLinker
+      return Proto3Codec.read(ReadBuffer.wrap(span, 0), out);
+    }
+
+    @Override public boolean decodeList(byte[] spans, Collection<Span> out) { // ex getTrace
+      return Proto3Codec.readList(ReadBuffer.wrap(spans, 0), out);
+    }
+
+    @Override public boolean decodeList(ByteBuffer spans, Collection<Span> out) {
+      return Proto3Codec.readList(ReadBuffer.wrapUnsafe(spans), out);
     }
 
-    @Override
-    public boolean decodeList(byte[] spans, Collection<Span> out) { // ex getTrace
-      return Proto3Codec.readList(spans, out);
+    @Override @Nullable public Span decodeOne(byte[] span) {
+      return Proto3Codec.readOne(ReadBuffer.wrap(span, 0));
     }
 
-    @Override
-    @Nullable
-    public Span decodeOne(byte[] span) {
-      return Proto3Codec.readOne(span);
+    @Override @Nullable public Span decodeOne(ByteBuffer span) {
+      return Proto3Codec.readOne(ReadBuffer.wrapUnsafe(span));
     }
 
-    @Override
-    public List<Span> decodeList(byte[] spans) {
+    @Override public List<Span> decodeList(byte[] spans) {
+      return decodeList(this, spans);
+    }
+
+    @Override public List<Span> decodeList(ByteBuffer spans) {
       return decodeList(this, spans);
     }
   };
 
+  /**
+   * ByteBuffer implementation of {@link #decodeList(byte[])}.
+   *
+   * <p>Note: only use this when it is ok to modify the underlying {@link ByteBuffer#array()}.
+   */
+  public abstract boolean decodeList(ByteBuffer spans, Collection<Span> out);
+
+  /**
+   * ByteBuffer implementation of {@link #decodeList(byte[])}.
+   *
+   * <p>Note: only use this when it is ok to modify the underlying {@link ByteBuffer#array()}.
+   */
+  public abstract List<Span> decodeList(ByteBuffer spans);
+
+  /**
+   * ByteBuffer implementation of {@link #decodeOne(byte[])}
+   *
+   * <p>Note: only use this when it is ok to modify the underlying {@link ByteBuffer#array()}.
+   */
+  @Nullable public abstract Span decodeOne(ByteBuffer span);
+
   static List<Span> decodeList(SpanBytesDecoder decoder, byte[] spans) {
     List<Span> out = new ArrayList<>();
     if (!decoder.decodeList(spans, out)) return Collections.emptyList();
     return out;
   }
+
+  static List<Span> decodeList(SpanBytesDecoder decoder, ByteBuffer spans) {
+    List<Span> out = new ArrayList<>();
+    if (!decoder.decodeList(spans, out)) return Collections.emptyList();
+    return out;
+  }
 }
diff --git a/zipkin/src/main/java/zipkin2/internal/Dependencies.java b/zipkin/src/main/java/zipkin2/internal/Dependencies.java
index 6141ed5..edabf75 100644
--- a/zipkin/src/main/java/zipkin2/internal/Dependencies.java
+++ b/zipkin/src/main/java/zipkin2/internal/Dependencies.java
@@ -27,7 +27,7 @@ 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;
+import static zipkin2.internal.WriteBuffer.utf8SizeInBytes;
 
 /**
  * Internal as only cassandra serializes the start and end timestamps along with link data, and
@@ -45,29 +45,30 @@ public final class Dependencies {
     return links;
   }
 
-  /** Reads from bytes serialized in TBinaryProtocol */
+  /** Reads from buffer serialized in TBinaryProtocol */
   public static Dependencies fromThrift(ByteBuffer bytes) {
     long startTs = 0L;
     long endTs = 0L;
     List<DependencyLink> links = Collections.emptyList();
 
+    ReadBuffer buffer = ReadBuffer.wrapUnsafe(bytes);
     while (true) {
-      ThriftField thriftField = ThriftField.read(bytes);
+      ThriftField thriftField = ThriftField.read(buffer);
       if (thriftField.type == TYPE_STOP) break;
 
       if (thriftField.isEqualTo(START_TS)) {
-        startTs = bytes.getLong();
+        startTs = buffer.readLong();
       } else if (thriftField.isEqualTo(END_TS)) {
-        endTs = bytes.getLong();
+        endTs = buffer.readLong();
       } else if (thriftField.isEqualTo(LINKS)) {
-        int length = ThriftCodec.readListLength(bytes);
+        int length = ThriftCodec.readListLength(buffer);
         if (length == 0) continue;
         links = new ArrayList<>(length);
         for (int i = 0; i < length; i++) {
-          links.add(DependencyLinkAdapter.read(bytes));
+          links.add(DependencyLinkAdapter.read(buffer));
         }
       } else {
-        skip(bytes, thriftField.type);
+        skip(buffer, thriftField.type);
       }
     }
 
@@ -76,9 +77,9 @@ public final class Dependencies {
 
   /** Writes the current instance in TBinaryProtocol */
   public ByteBuffer toThrift() {
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(sizeInBytes());
-    write(buffer);
-    return ByteBuffer.wrap(buffer.unwrap());
+    byte[] result = new byte[sizeInBytes()];
+    write(WriteBuffer.wrap(result, 0));
+    return ByteBuffer.wrap(result);
   }
 
   int sizeInBytes() {
@@ -90,7 +91,7 @@ public final class Dependencies {
     return sizeInBytes;
   }
 
-  void write(UnsafeBuffer buffer) {
+  void write(WriteBuffer buffer) {
     START_TS.write(buffer);
     ThriftCodec.writeLong(buffer, startTs);
 
@@ -138,39 +139,38 @@ public final class Dependencies {
     return h;
   }
 
-  static final class DependencyLinkAdapter implements UnsafeBuffer.Writer<DependencyLink> {
+  static final class DependencyLinkAdapter implements WriteBuffer.Writer<DependencyLink> {
 
     static final ThriftField PARENT = new ThriftField(TYPE_STRING, 1);
     static final ThriftField CHILD = new ThriftField(TYPE_STRING, 2);
     static final ThriftField CALL_COUNT = new ThriftField(TYPE_I64, 4);
     static final ThriftField ERROR_COUNT = new ThriftField(TYPE_I64, 5);
 
-    static DependencyLink read(ByteBuffer bytes) {
+    static DependencyLink read(ReadBuffer buffer) {
       DependencyLink.Builder result = DependencyLink.newBuilder();
       ThriftField thriftField;
 
       while (true) {
-        thriftField = ThriftField.read(bytes);
+        thriftField = ThriftField.read(buffer);
         if (thriftField.type == TYPE_STOP) break;
 
         if (thriftField.isEqualTo(PARENT)) {
-          result.parent(ThriftCodec.readUtf8(bytes));
+          result.parent(buffer.readUtf8(buffer.readInt()));
         } else if (thriftField.isEqualTo(CHILD)) {
-          result.child(ThriftCodec.readUtf8(bytes));
+          result.child(buffer.readUtf8(buffer.readInt()));
         } else if (thriftField.isEqualTo(CALL_COUNT)) {
-          result.callCount(bytes.getLong());
+          result.callCount(buffer.readLong());
         } else if (thriftField.isEqualTo(ERROR_COUNT)) {
-          result.errorCount(bytes.getLong());
+          result.errorCount(buffer.readLong());
         } else {
-          skip(bytes, thriftField.type);
+          skip(buffer, thriftField.type);
         }
       }
 
       return result.build();
     }
 
-    @Override
-    public int sizeInBytes(DependencyLink value) {
+    @Override public int sizeInBytes(DependencyLink value) {
       int sizeInBytes = 0;
       sizeInBytes += 3 + 4 + utf8SizeInBytes(value.parent());
       sizeInBytes += 3 + 4 + utf8SizeInBytes(value.child());
@@ -180,8 +180,7 @@ public final class Dependencies {
       return sizeInBytes;
     }
 
-    @Override
-    public void write(DependencyLink value, UnsafeBuffer buffer) {
+    @Override public void write(DependencyLink value, WriteBuffer buffer) {
       PARENT.write(buffer);
       ThriftCodec.writeLengthPrefixed(buffer, value.parent());
 
diff --git a/zipkin/src/main/java/zipkin2/internal/HexCodec.java b/zipkin/src/main/java/zipkin2/internal/HexCodec.java
index e90e0a1..0739b3b 100644
--- a/zipkin/src/main/java/zipkin2/internal/HexCodec.java
+++ b/zipkin/src/main/java/zipkin2/internal/HexCodec.java
@@ -18,6 +18,10 @@ package zipkin2.internal;
 
 // code originally imported from zipkin.Util
 public final class HexCodec {
+  public static final char[] HEX_DIGITS = {
+    '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'
+  };
+
   /**
    * Parses a 1 to 32 character lower-hex string with no prefix into an unsigned long, tossing any
    * bits higher than 64.
diff --git a/zipkin/src/main/java/zipkin2/internal/JsonCodec.java b/zipkin/src/main/java/zipkin2/internal/JsonCodec.java
index 0ecf4cc..583c683 100644
--- a/zipkin/src/main/java/zipkin2/internal/JsonCodec.java
+++ b/zipkin/src/main/java/zipkin2/internal/JsonCodec.java
@@ -16,7 +16,6 @@
  */
 package zipkin2.internal;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.charset.Charset;
@@ -30,7 +29,8 @@ import static com.google.gson.stream.JsonToken.STRING;
 import static java.lang.String.format;
 
 /**
- * This explicitly constructs instances of model classes via manual parsing for a number of reasons.
+ * This explicitly constructs instances of model classes via manual parsing for a number of
+ * reasons.
  *
  * <ul>
  *   <li>Eliminates the need to keep separate model classes for proto3 vs json
@@ -44,14 +44,14 @@ import static java.lang.String.format;
  * this should be easy to justify as these objects don't change much at all.
  */
 public final class JsonCodec {
+  static final Charset UTF_8 = Charset.forName("UTF-8");
+
   // Hides gson types for internal use in other submodules
   public static final class JsonReader {
     final com.google.gson.stream.JsonReader delegate;
 
-    JsonReader(byte[] bytes) {
-      delegate =
-        new com.google.gson.stream.JsonReader(
-          new InputStreamReader(new ByteArrayInputStream(bytes), UTF_8));
+    JsonReader(ReadBuffer buffer) {
+      delegate = new com.google.gson.stream.JsonReader(new InputStreamReader(buffer, UTF_8));
     }
 
     public void beginArray() throws IOException {
@@ -114,38 +114,36 @@ public final class JsonCodec {
       return delegate.peek() == NULL;
     }
 
-    @Override
-    public String toString() {
+    @Override public String toString() {
       return delegate.toString();
     }
   }
 
-  static final Charset UTF_8 = Charset.forName("UTF-8");
-
   public interface JsonReaderAdapter<T> {
     T fromJson(JsonReader reader) throws IOException;
   }
 
-  public static <T> boolean read(JsonReaderAdapter<T> adapter, byte[] bytes, Collection<T> out) {
-    if (bytes.length == 0) return false;
+  public static <T> boolean read(
+    JsonReaderAdapter<T> adapter, ReadBuffer buffer, Collection<T> out) {
+    if (buffer.available() == 0) return false;
     try {
-      out.add(adapter.fromJson(new JsonReader(bytes)));
+      out.add(adapter.fromJson(new JsonReader(buffer)));
       return true;
     } catch (Exception e) {
       throw exceptionReading(adapter.toString(), e);
     }
   }
 
-  public static @Nullable <T> T readOne(JsonReaderAdapter<T> adapter, byte[] bytes) {
+  public static @Nullable <T> T readOne(JsonReaderAdapter<T> adapter, ReadBuffer buffer) {
     List<T> out = new ArrayList<>(1); // TODO: could make single-element list w/o array
-    if (!read(adapter, bytes, out)) return null;
+    if (!read(adapter, buffer, out)) return null;
     return out.get(0);
   }
 
   public static <T> boolean readList(
-    JsonReaderAdapter<T> adapter, byte[] bytes, Collection<T> out) {
-    if (bytes.length == 0) return false;
-    JsonReader reader = new JsonReader(bytes);
+    JsonReaderAdapter<T> adapter, ReadBuffer buffer, Collection<T> out) {
+    if (buffer.available() == 0) return false;
+    JsonReader reader = new JsonReader(buffer);
     try {
       reader.beginArray();
       if (!reader.hasNext()) return false;
@@ -157,7 +155,7 @@ public final class JsonCodec {
     }
   }
 
-  static <T> int sizeInBytes(UnsafeBuffer.Writer<T> writer, List<T> value) {
+  static <T> int sizeInBytes(WriteBuffer.Writer<T> writer, List<T> value) {
     int length = value.size();
     int sizeInBytes = 2; // []
     if (length > 1) sizeInBytes += length - 1; // comma to join elements
@@ -168,15 +166,15 @@ public final class JsonCodec {
   }
 
   /** Inability to encode is a programming bug. */
-  public static <T> byte[] write(UnsafeBuffer.Writer<T> writer, T value) {
-    UnsafeBuffer b = UnsafeBuffer.allocate(writer.sizeInBytes(value));
+  public static <T> byte[] write(WriteBuffer.Writer<T> writer, T value) {
+    byte[] result = new byte[writer.sizeInBytes(value)];
+    WriteBuffer b = WriteBuffer.wrap(result, 0);
     try {
       writer.write(value, b);
     } catch (RuntimeException e) {
-      byte[] bytes = b.unwrap();
-      int lengthWritten = bytes.length;
-      for (int i = 0; i < bytes.length; i++) {
-        if (bytes[i] == 0) {
+      int lengthWritten = result.length;
+      for (int i = 0; i < result.length; i++) {
+        if (result[i] == 0) {
           lengthWritten = i;
           break;
         }
@@ -190,33 +188,34 @@ public final class JsonCodec {
           writer.getClass().getSimpleName(),
           value.getClass().getSimpleName(),
           lengthWritten,
-          bytes.length,
-          new String(bytes, 0, lengthWritten, UTF_8));
+          result.length,
+          new String(result, 0, lengthWritten, UTF_8));
       throw Platform.get().assertionError(message, e);
     }
-    return b.unwrap();
+    return result;
   }
 
-  public static <T> byte[] writeList(UnsafeBuffer.Writer<T> writer, List<T> value) {
+  public static <T> byte[] writeList(WriteBuffer.Writer<T> writer, List<T> value) {
     if (value.isEmpty()) return new byte[] {'[', ']'};
-    UnsafeBuffer result = UnsafeBuffer.allocate(sizeInBytes(writer, value));
-    writeList(writer, value, result);
-    return result.unwrap();
+    byte[] result = new byte[sizeInBytes(writer, value)];
+    writeList(writer, value, WriteBuffer.wrap(result, 0));
+    return result;
   }
 
-  public static <T> int writeList(UnsafeBuffer.Writer<T> writer, List<T> value, byte[] out, int pos) {
+  public static <T> int writeList(WriteBuffer.Writer<T> writer, List<T> value, byte[] out,
+    int pos) {
     if (value.isEmpty()) {
       out[pos++] = '[';
       out[pos++] = ']';
       return 2;
     }
     int initialPos = pos;
-    UnsafeBuffer result = UnsafeBuffer.wrap(out, pos);
+    WriteBuffer result = WriteBuffer.wrap(out, pos);
     writeList(writer, value, result);
     return result.pos() - initialPos;
   }
 
-  public static <T> void writeList(UnsafeBuffer.Writer<T> writer, List<T> value, UnsafeBuffer b) {
+  public static <T> void writeList(WriteBuffer.Writer<T> writer, List<T> value, WriteBuffer 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 77f0093..f28cc7b 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 UnsafeBuffer.utf8SizeInBytes(v) + escapingOverhead;
+    return WriteBuffer.utf8SizeInBytes(v) + escapingOverhead;
   }
 }
diff --git a/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java b/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java
index 52d646e..add62ed 100644
--- a/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java
+++ b/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java
@@ -44,9 +44,8 @@ public final class Proto3Codec {
     return writer.writeList(spans, out, pos);
   }
 
-  public static boolean read(byte[] bytes, Collection<Span> out) {
-    if (bytes.length == 0) return false;
-    UnsafeBuffer buffer = UnsafeBuffer.wrap(bytes, 0);
+  public static boolean read(ReadBuffer buffer, Collection<Span> out) {
+    if (buffer.available() == 0) return false;
     try {
       Span span = SPAN.read(buffer);
       if (span == null) return false;
@@ -57,18 +56,17 @@ public final class Proto3Codec {
     }
   }
 
-  public static @Nullable Span readOne(byte[] bytes) {
+  public static @Nullable Span readOne(ReadBuffer buffer) {
     try {
-      return SPAN.read(UnsafeBuffer.wrap(bytes, 0));
+      return SPAN.read(buffer);
     } catch (RuntimeException e) {
       throw exceptionReading("Span", e);
     }
   }
 
-  public static boolean readList(byte[] bytes, Collection<Span> out) {
-    int length = bytes.length;
+  public static boolean readList(ReadBuffer buffer, Collection<Span> out) {
+    int length = buffer.available();
     if (length == 0) return false;
-    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 b08ce90..ab199ed 100644
--- a/zipkin/src/main/java/zipkin2/internal/Proto3Fields.java
+++ b/zipkin/src/main/java/zipkin2/internal/Proto3Fields.java
@@ -18,6 +18,9 @@ package zipkin2.internal;
 
 import zipkin2.Endpoint;
 
+import static zipkin2.internal.WriteBuffer.utf8SizeInBytes;
+import static zipkin2.internal.WriteBuffer.varintSizeInBytes;
+
 /**
  * Everything here assumes the field numbers are less than 16, implying a 1 byte tag.
  */
@@ -70,8 +73,8 @@ final class Proto3Fields {
       return wireType;
     }
 
-    static boolean skipValue(UnsafeBuffer buffer, int wireType) {
-      int remaining = buffer.remaining();
+    static boolean skipValue(ReadBuffer buffer, int wireType) {
+      int remaining = buffer.available();
       switch (wireType) {
         case WIRETYPE_VARINT:
           for (int i = 0; i < remaining; i++) {
@@ -79,12 +82,12 @@ final class Proto3Fields {
           }
           return false;
         case WIRETYPE_FIXED64:
-          return buffer.skip(8);
+          return buffer.skip(8) == 8;
         case WIRETYPE_LENGTH_DELIMITED:
           int length = buffer.readVarint32();
-          return buffer.skip(length);
+          return buffer.skip(length) == length;
         case WIRETYPE_FIXED32:
-          return buffer.skip(4);
+          return buffer.skip(4) == 4;
         default:
           throw new IllegalArgumentException(
             "Malformed: invalid wireType " + wireType + " at byte " + buffer.pos());
@@ -111,7 +114,7 @@ final class Proto3Fields {
       return sizeOfLengthDelimitedField(sizeOfValue);
     }
 
-    final void write(UnsafeBuffer b, T value) {
+    final void write(WriteBuffer b, T value) {
       if (value == null) return;
       int sizeOfValue = sizeOfValue(value);
       b.writeByte(key);
@@ -123,7 +126,7 @@ 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(UnsafeBuffer b) {
+    final T readLengthPrefixAndValue(ReadBuffer b) {
       int length = b.readVarint32();
       if (length == 0) return null;
       return readValue(b, length);
@@ -131,10 +134,10 @@ final class Proto3Fields {
 
     abstract int sizeOfValue(T value);
 
-    abstract void writeValue(UnsafeBuffer b, T value);
+    abstract void writeValue(WriteBuffer b, T value);
 
     /** @param length is greater than zero */
-    abstract T readValue(UnsafeBuffer b, int length);
+    abstract T readValue(ReadBuffer b, int length);
   }
 
   static class BytesField extends LengthDelimitedField<byte[]> {
@@ -146,11 +149,11 @@ final class Proto3Fields {
       return bytes.length;
     }
 
-    @Override void writeValue(UnsafeBuffer b, byte[] bytes) {
+    @Override void writeValue(WriteBuffer b, byte[] bytes) {
       b.write(bytes);
     }
 
-    @Override byte[] readValue(UnsafeBuffer b, int length) {
+    @Override byte[] readValue(ReadBuffer b, int length) {
       return b.readBytes(length);
     }
   }
@@ -165,7 +168,7 @@ final class Proto3Fields {
       return hex.length() / 2;
     }
 
-    @Override void writeValue(UnsafeBuffer b, String hex) {
+    @Override void writeValue(WriteBuffer 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;
@@ -180,7 +183,7 @@ final class Proto3Fields {
       throw new AssertionError("not lowerHex " + c); // bug
     }
 
-    @Override String readValue(UnsafeBuffer buffer, int length) {
+    @Override String readValue(ReadBuffer buffer, int length) {
       return buffer.readBytesAsHex(length);
     }
   }
@@ -191,14 +194,14 @@ final class Proto3Fields {
     }
 
     @Override int sizeOfValue(String utf8) {
-      return utf8 != null ? UnsafeBuffer.utf8SizeInBytes(utf8) : 0;
+      return utf8 != null ? utf8SizeInBytes(utf8) : 0;
     }
 
-    @Override void writeValue(UnsafeBuffer b, String utf8) {
+    @Override void writeValue(WriteBuffer b, String utf8) {
       b.writeUtf8(utf8);
     }
 
-    @Override String readValue(UnsafeBuffer buffer, int length) {
+    @Override String readValue(ReadBuffer buffer, int length) {
       return buffer.readUtf8(length);
     }
   }
@@ -209,7 +212,7 @@ final class Proto3Fields {
       assert wireType == WIRETYPE_FIXED64;
     }
 
-    void write(UnsafeBuffer b, long number) {
+    void write(WriteBuffer b, long number) {
       if (number == 0) return;
       b.writeByte(key);
       b.writeLongLe(number);
@@ -220,7 +223,7 @@ final class Proto3Fields {
       return 1 + 8; // tag + 8 byte number
     }
 
-    long readValue(UnsafeBuffer buffer) {
+    long readValue(ReadBuffer buffer) {
       return buffer.readLongLe();
     }
   }
@@ -232,20 +235,20 @@ final class Proto3Fields {
     }
 
     int sizeInBytes(int number) {
-      return number != 0 ? 1 + UnsafeBuffer.varintSizeInBytes(number) : 0; // tag + varint
+      return number != 0 ? 1 + varintSizeInBytes(number) : 0; // tag + varint
     }
 
-    void write(UnsafeBuffer b, int number) {
+    void write(WriteBuffer b, int number) {
       if (number == 0) return;
       b.writeByte(key);
       b.writeVarint(number);
     }
 
     int sizeInBytes(long number) {
-      return number != 0 ? 1 + UnsafeBuffer.varintSizeInBytes(number) : 0; // tag + varint
+      return number != 0 ? 1 + varintSizeInBytes(number) : 0; // tag + varint
     }
 
-    void write(UnsafeBuffer b, long number) {
+    void write(WriteBuffer b, long number) {
       if (number == 0) return;
       b.writeByte(key);
       b.writeVarint(number);
@@ -262,13 +265,13 @@ final class Proto3Fields {
       return bool ? 2 : 0; // tag + varint
     }
 
-    void write(UnsafeBuffer b, boolean bool) {
+    void write(WriteBuffer b, boolean bool) {
       if (!bool) return;
       b.writeByte(key);
       b.writeByte(1);
     }
 
-    boolean read(UnsafeBuffer b) {
+    boolean read(ReadBuffer b) {
       byte bool = b.readByte();
       if (bool < 0 || bool > 1) {
         throw new IllegalArgumentException("Malformed: invalid boolean value at byte " + b.pos());
@@ -291,6 +294,6 @@ final class Proto3Fields {
   }
 
   static int sizeOfLengthDelimitedField(int sizeInBytes) {
-    return 1 + UnsafeBuffer.varintSizeInBytes(sizeInBytes) + sizeInBytes; // tag + len + bytes
+    return 1 + 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 8e2d8cd..b118a54 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 UnsafeBuffer.Writer<Span> {
+final class Proto3SpanWriter implements WriteBuffer.Writer<Span> {
 
   static final byte[] EMPTY_ARRAY = new byte[0];
 
@@ -31,7 +31,7 @@ final class Proto3SpanWriter implements UnsafeBuffer.Writer<Span> {
     return SPAN.sizeInBytes(span);
   }
 
-  @Override public void write(Span value, UnsafeBuffer b) {
+  @Override public void write(Span value, WriteBuffer b) {
     SPAN.write(b, value);
   }
 
@@ -51,22 +51,23 @@ final class Proto3SpanWriter implements UnsafeBuffer.Writer<Span> {
       int sizeOfValue = sizeOfValues[i] = SPAN.sizeOfValue(spans.get(i));
       sizeInBytes += sizeOfLengthDelimitedField(sizeOfValue);
     }
-    UnsafeBuffer result = UnsafeBuffer.allocate(sizeInBytes);
+    byte[] result = new byte[sizeInBytes];
+    WriteBuffer writeBuffer = WriteBuffer.wrap(result, 0);
     for (int i = 0; i < lengthOfSpans; i++) {
-      writeSpan(spans.get(i), sizeOfValues[i], result);
+      writeSpan(spans.get(i), sizeOfValues[i], writeBuffer);
     }
-    return result.unwrap();
+    return result;
   }
 
   byte[] write(Span onlySpan) {
     int sizeOfValue = SPAN.sizeOfValue(onlySpan);
-    UnsafeBuffer result = UnsafeBuffer.allocate(sizeOfLengthDelimitedField(sizeOfValue));
-    writeSpan(onlySpan, sizeOfValue, result);
-    return result.unwrap();
+    byte[] result = new byte[sizeOfLengthDelimitedField(sizeOfValue)];
+    writeSpan(onlySpan, sizeOfValue, WriteBuffer.wrap(result, 0));
+    return result;
   }
 
   // prevents resizing twice
-  void writeSpan(Span span, int sizeOfSpan, UnsafeBuffer result) {
+  void writeSpan(Span span, int sizeOfSpan, WriteBuffer result) {
     result.writeByte(SPAN.key);
     result.writeVarint(sizeOfSpan); // length prefix
     SPAN.writeValue(result, span);
@@ -76,7 +77,7 @@ final class Proto3SpanWriter implements UnsafeBuffer.Writer<Span> {
     int lengthOfSpans = spans.size();
     if (lengthOfSpans == 0) return 0;
 
-    UnsafeBuffer result = UnsafeBuffer.wrap(out, pos);
+    WriteBuffer result = WriteBuffer.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 5d273ef..a5c94ea 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(UnsafeBuffer b, Endpoint value) {
+    @Override void writeValue(WriteBuffer 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(UnsafeBuffer buffer, int length) {
+    @Override Endpoint readValue(ReadBuffer 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(UnsafeBuffer b, int length) {
+    @Override final T readValue(ReadBuffer b, int length) {
       throw new UnsupportedOperationException();
     }
 
-    abstract boolean readLengthPrefixAndValue(UnsafeBuffer b, Span.Builder builder);
+    abstract boolean readLengthPrefixAndValue(ReadBuffer 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(UnsafeBuffer b, Annotation value) {
+    @Override void writeValue(WriteBuffer b, Annotation value) {
       TIMESTAMP.write(b, value.timestamp());
       VALUE.write(b, value.value());
     }
 
-    @Override boolean readLengthPrefixAndValue(UnsafeBuffer b, Span.Builder builder) {
+    @Override boolean readLengthPrefixAndValue(ReadBuffer b, Span.Builder builder) {
       int length = b.readVarint32();
       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(UnsafeBuffer b, Map.Entry<String, String> value) {
+    @Override void writeValue(WriteBuffer b, Map.Entry<String, String> value) {
       KEY.write(b, value.getKey());
       VALUE.write(b, value.getValue());
     }
 
-    @Override boolean readLengthPrefixAndValue(UnsafeBuffer b, Span.Builder builder) {
+    @Override boolean readLengthPrefixAndValue(ReadBuffer b, Span.Builder builder) {
       int length = b.readVarint32();
       if (length == 0) return false;
       int endPos = b.pos() + length;
@@ -276,7 +276,7 @@ final class Proto3ZipkinFields {
       return sizeOfSpan;
     }
 
-    @Override void writeValue(UnsafeBuffer b, Span value) {
+    @Override void writeValue(WriteBuffer 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(UnsafeBuffer buffer) {
+    public Span read(ReadBuffer buffer) {
       buffer.readVarint32(); // toss the key
       return readLengthPrefixAndValue(buffer);
     }
 
-    @Override Span readValue(UnsafeBuffer buffer, int length) {
+    @Override Span readValue(ReadBuffer buffer, int length) {
       buffer.require(length); // more convenient to check up-front vs partially read
       int endPos = buffer.pos() + length;
 
@@ -373,7 +373,7 @@ final class Proto3ZipkinFields {
     }
   }
 
-  static void logAndSkip(UnsafeBuffer buffer, int nextKey) {
+  static void logAndSkip(ReadBuffer 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 258c5a5..1d1abe9 100644
--- a/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java
+++ b/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java
@@ -18,8 +18,6 @@ package zipkin2.internal;
 
 import java.io.EOFException;
 import java.nio.BufferUnderflowException;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -39,11 +37,9 @@ 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 {
-  static final Charset UTF_8 = Charset.forName("UTF-8");
   // break vs recursing infinitely when skipping data
   static final int MAX_SKIP_DEPTH = 2147483647;
 
@@ -58,7 +54,7 @@ public final class ThriftCodec {
   }
 
   /** Encoding overhead is thrift type plus 32-bit length prefix */
-  static <T> int listSizeInBytes(UnsafeBuffer.Writer<T> writer, List<T> values) {
+  static <T> int listSizeInBytes(WriteBuffer.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));
@@ -66,9 +62,8 @@ public final class ThriftCodec {
     return sizeInBytes;
   }
 
-  public static boolean read(byte[] bytes, Collection<Span> out) {
-    if (bytes.length == 0) return false;
-    ByteBuffer buffer = ByteBuffer.wrap(bytes);
+  public static boolean read(ReadBuffer buffer, Collection<Span> out) {
+    if (buffer.available() == 0) return false;
     try {
       V1Span v1Span = new V1ThriftSpanReader().read(buffer);
       V1SpanConverter.create().convert(v1Span, out);
@@ -79,9 +74,8 @@ public final class ThriftCodec {
   }
 
   @Nullable
-  public static Span readOne(byte[] bytes) {
-    if (bytes.length == 0) return null;
-    ByteBuffer buffer = ByteBuffer.wrap(bytes);
+  public static Span readOne(ReadBuffer buffer) {
+    if (buffer.available() == 0) return null;
     try {
       V1Span v1Span = new V1ThriftSpanReader().read(buffer);
       List<Span> out = new ArrayList<>(1);
@@ -92,10 +86,9 @@ public final class ThriftCodec {
     }
   }
 
-  public static boolean readList(byte[] bytes, Collection<Span> out) {
-    int length = bytes.length;
+  public static boolean readList(ReadBuffer buffer, Collection<Span> out) {
+    int length = buffer.available();
     if (length == 0) return false;
-    ByteBuffer buffer = ByteBuffer.wrap(bytes);
     try {
       int listLength = readListLength(buffer);
       if (listLength == 0) return false;
@@ -111,12 +104,12 @@ public final class ThriftCodec {
     return true;
   }
 
-  static int readListLength(ByteBuffer bytes) {
-    byte ignoredType = bytes.get();
-    return guardLength(bytes);
+  static int readListLength(ReadBuffer buffer) {
+    byte ignoredType = buffer.readByte();
+    return buffer.readInt();
   }
 
-  static <T> void writeList(UnsafeBuffer.Writer<T> writer, List<T> value, UnsafeBuffer buffer) {
+  static <T> void writeList(WriteBuffer.Writer<T> writer, List<T> value, WriteBuffer buffer) {
     int length = value.size();
     writeListBegin(buffer, length);
     for (int i = 0; i < length; i++) {
@@ -134,50 +127,49 @@ public final class ThriftCodec {
     throw new IllegalArgumentException(message, e);
   }
 
-  static void skip(ByteBuffer bytes, byte type) {
-    skip(bytes, type, MAX_SKIP_DEPTH);
+  static void skip(ReadBuffer buffer, byte type) {
+    skip(buffer, type, MAX_SKIP_DEPTH);
   }
 
-  static void skip(ByteBuffer bytes, byte type, int maxDepth) {
+  static void skip(ReadBuffer buffer, byte type, int maxDepth) {
     if (maxDepth <= 0) throw new IllegalStateException("Maximum skip depth exceeded");
     switch (type) {
       case TYPE_BOOL:
       case TYPE_BYTE:
-        skip(bytes, 1);
+        buffer.skip(1);
         break;
       case TYPE_I16:
-        skip(bytes, 2);
+        buffer.skip(2);
         break;
       case TYPE_I32:
-        skip(bytes, 4);
+        buffer.skip(4);
         break;
       case TYPE_DOUBLE:
       case TYPE_I64:
-        skip(bytes, 8);
+        buffer.skip(8);
         break;
       case TYPE_STRING:
-        int size = guardLength(bytes);
-        skip(bytes, size);
+        buffer.skip(buffer.readInt());
         break;
       case TYPE_STRUCT:
         while (true) {
-          ThriftField thriftField = ThriftField.read(bytes);
+          ThriftField thriftField = ThriftField.read(buffer);
           if (thriftField.type == TYPE_STOP) return;
-          skip(bytes, thriftField.type, maxDepth - 1);
+          skip(buffer, thriftField.type, maxDepth - 1);
         }
       case TYPE_MAP:
-        byte keyType = bytes.get();
-        byte valueType = bytes.get();
-        for (int i = 0, length = guardLength(bytes); i < length; i++) {
-          skip(bytes, keyType, maxDepth - 1);
-          skip(bytes, valueType, maxDepth - 1);
+        byte keyType = buffer.readByte();
+        byte valueType = buffer.readByte();
+        for (int i = 0, length = buffer.readInt(); i < length; i++) {
+          skip(buffer, keyType, maxDepth - 1);
+          skip(buffer, valueType, maxDepth - 1);
         }
         break;
       case TYPE_SET:
       case TYPE_LIST:
-        byte elemType = bytes.get();
-        for (int i = 0, length = guardLength(bytes); i < length; i++) {
-          skip(bytes, elemType, maxDepth - 1);
+        byte elemType = buffer.readByte();
+        for (int i = 0, length = buffer.readInt(); i < length; i++) {
+          skip(buffer, elemType, maxDepth - 1);
         }
         break;
       default: // types that don't need explicit skipping
@@ -185,83 +177,25 @@ public final class ThriftCodec {
     }
   }
 
-  static void skip(ByteBuffer bytes, int count) {
-    // avoid java.lang.NoSuchMethodError: java.nio.ByteBuffer.position(I)Ljava/nio/ByteBuffer;
-    // bytes.position(bytes.position() + count);
-    for (int i = 0; i < count && bytes.hasRemaining(); i++) {
-      bytes.get();
-    }
-  }
-
-  static byte[] readByteArray(ByteBuffer bytes) {
-    return readByteArray(bytes, guardLength(bytes));
-  }
-
-  static final String ONE = Character.toString((char) 1);
-
-  static byte[] readByteArray(ByteBuffer bytes, int length) {
-    byte[] copy = new byte[length];
-    if (!bytes.hasArray()) {
-      bytes.get(copy);
-      return copy;
-    }
-
-    byte[] original = bytes.array();
-    int offset = bytes.arrayOffset() + bytes.position();
-    System.arraycopy(original, offset, copy, 0, length);
-    bytes.position(bytes.position() + length);
-    return copy;
-  }
-
-  static String readUtf8(ByteBuffer bytes) {
-    int length = guardLength(bytes);
-    if (length == 0) return ""; // ex empty name
-    if (length == 1) {
-      byte single = bytes.get();
-      if (single == 1) return ONE; // special case for address annotations
-      return Character.toString((char) single);
-    }
-
-    if (!bytes.hasArray()) return new String(readByteArray(bytes, length), UTF_8);
-
-    int offset = bytes.arrayOffset() + bytes.position();
-    String result = UnsafeBuffer.wrap(bytes.array(), offset).readUtf8(length);
-    bytes.position(bytes.position() + length);
-    return result;
-  }
-
-  static int guardLength(ByteBuffer buffer) {
-    int length = buffer.getInt();
-    guardLength(buffer, length);
-    return length;
-  }
-
-  static void guardLength(ByteBuffer buffer, int length) {
-    if (length > buffer.remaining()) {
-      throw new IllegalArgumentException(
-        "Truncated: length " + length + " > bytes remaining " + buffer.remaining());
-    }
-  }
-
-  static void writeListBegin(UnsafeBuffer buffer, int size) {
+  static void writeListBegin(WriteBuffer buffer, int size) {
     buffer.writeByte(TYPE_STRUCT);
     writeInt(buffer, size);
   }
 
-  static void writeLengthPrefixed(UnsafeBuffer buffer, String utf8) {
-    int ignoredLength = utf8SizeInBytes(utf8);
-    writeInt(buffer, utf8SizeInBytes(utf8));
+  static void writeLengthPrefixed(WriteBuffer buffer, String utf8) {
+    int ignoredLength = WriteBuffer.utf8SizeInBytes(utf8);
+    writeInt(buffer, WriteBuffer.utf8SizeInBytes(utf8));
     buffer.writeUtf8(utf8);
   }
 
-  static void writeInt(UnsafeBuffer buf, int v) {
+  static void writeInt(WriteBuffer 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(UnsafeBuffer buf, long v) {
+  static void writeLong(WriteBuffer 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 fd646ca..c2527d2 100644
--- a/zipkin/src/main/java/zipkin2/internal/ThriftEndpointCodec.java
+++ b/zipkin/src/main/java/zipkin2/internal/ThriftEndpointCodec.java
@@ -16,16 +16,14 @@
  */
 package zipkin2.internal;
 
-import java.nio.ByteBuffer;
 import zipkin2.Endpoint;
 
-import static zipkin2.internal.ThriftCodec.guardLength;
 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;
+import static zipkin2.internal.WriteBuffer.utf8SizeInBytes;
 
 final class ThriftEndpointCodec {
   static final byte[] INT_ZERO = {0, 0, 0, 0};
@@ -34,16 +32,15 @@ final class ThriftEndpointCodec {
   static final ThriftField SERVICE_NAME = new ThriftField(TYPE_STRING, 3);
   static final ThriftField IPV6 = new ThriftField(TYPE_STRING, 4);
 
-  static Endpoint read(ByteBuffer bytes) {
+  static Endpoint read(ReadBuffer buffer) {
     Endpoint.Builder result = Endpoint.newBuilder();
 
     while (true) {
-      ThriftField thriftField = ThriftField.read(bytes);
+      ThriftField thriftField = ThriftField.read(buffer);
       if (thriftField.type == TYPE_STOP) break;
 
       if (thriftField.isEqualTo(IPV4)) {
-        guardLength(bytes, 4);
-        int ipv4 = bytes.getInt();
+        int ipv4 = buffer.readInt();
         if (ipv4 != 0) {
           result.parseIp( // allocation is ok here as Endpoint.ipv4Bytes would anyway
             new byte[] {
@@ -54,14 +51,13 @@ final class ThriftEndpointCodec {
             });
         }
       } else if (thriftField.isEqualTo(PORT)) {
-        guardLength(bytes, 2);
-        result.port(bytes.getShort() & 0xFFFF);
+        result.port(buffer.readShort() & 0xFFFF);
       } else if (thriftField.isEqualTo(SERVICE_NAME)) {
-        result.serviceName(ThriftCodec.readUtf8(bytes));
+        result.serviceName(buffer.readUtf8(buffer.readInt()));
       } else if (thriftField.isEqualTo(IPV6)) {
-        result.parseIp(ThriftCodec.readByteArray(bytes));
+        result.parseIp(buffer.readBytes(buffer.readInt()));
       } else {
-        skip(bytes, thriftField.type);
+        skip(buffer, thriftField.type);
       }
     }
     return result.build();
@@ -78,7 +74,7 @@ final class ThriftEndpointCodec {
     return sizeInBytes;
   }
 
-  static void write(Endpoint value, UnsafeBuffer buffer) {
+  static void write(Endpoint value, WriteBuffer 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 e0d62b4..0088de1 100644
--- a/zipkin/src/main/java/zipkin2/internal/ThriftField.java
+++ b/zipkin/src/main/java/zipkin2/internal/ThriftField.java
@@ -16,8 +16,6 @@
  */
 package zipkin2.internal;
 
-import java.nio.ByteBuffer;
-
 final class ThriftField {
   // taken from org.apache.thrift.protocol.TType
   static final byte TYPE_STOP = 0;
@@ -41,16 +39,16 @@ final class ThriftField {
     this.id = id;
   }
 
-  void write(UnsafeBuffer buffer) {
+  void write(WriteBuffer buffer) {
     buffer.writeByte(type);
     // Write ID as a short!
     buffer.writeByte((id >>> 8L) & 0xff);
     buffer.writeByte(id & 0xff);
   }
 
-  static ThriftField read(ByteBuffer bytes) {
-    byte type = bytes.get();
-    return new ThriftField(type, type == TYPE_STOP ? TYPE_STOP : bytes.getShort());
+  static ThriftField read(ReadBuffer bytes) {
+    byte type = bytes.readByte();
+    return new ThriftField(type, type == TYPE_STOP ? TYPE_STOP : bytes.readShort());
   }
 
   boolean isEqualTo(ThriftField that) {
diff --git a/zipkin/src/main/java/zipkin2/internal/UnsafeBuffer.java b/zipkin/src/main/java/zipkin2/internal/UnsafeBuffer.java
deleted file mode 100644
index 0ae5d7f..0000000
--- a/zipkin/src/main/java/zipkin2/internal/UnsafeBuffer.java
+++ /dev/null
@@ -1,454 +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;
-
-import static zipkin2.internal.JsonCodec.UTF_8;
-
-/**
- * <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 final char[] HEX_DIGITS = {
-    '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'
-  };
-
-  public static UnsafeBuffer wrap(byte[] bytes, int pos) {
-    return new UnsafeBuffer(bytes, pos);
-  }
-
-  public static UnsafeBuffer allocate(int sizeInBytes) {
-    return new UnsafeBuffer(sizeInBytes);
-  }
-
-  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 = maybeDecodeShortAsciiString(buf, pos, length);
-    if (result == null) result = new String(buf, pos, length, UTF_8);
-    pos += length;
-    return result;
-  }
-
-  // Speculatively assume all 7-bit ASCII characters.. common in normal tags and names
-  @Nullable static String maybeDecodeShortAsciiString(byte[] buf, int offset, int length) {
-    if (length == 0) return ""; // ex error tag with no value
-    if (length > Platform.SHORT_STRING_LENGTH) return null;
-    char[] buffer = Platform.shortStringBuffer();
-    for (int i = 0; i < length; i++) {
-      byte b = buf[offset + i];
-      if ((b & 0x80) != 0) return null; // Not 7-bit ASCII character
-      buffer[i] = (char) b;
-    }
-    return new String(buffer, 0, length);
-  }
-
-  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.shortStringBuffer();
-
-    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.
-   *
-   * <p>Originally based on
-   * http://stackoverflow.com/questions/8511490/calculating-length-in-utf-8-of-java-string-without-actually-encoding-it
-   *
-   * <p>Later, ASCII run and malformed surrogate logic borrowed from okio.Utf8
-   */
-  public static int utf8SizeInBytes(CharSequence string) {
-    int sizeInBytes = 0;
-    for (int i = 0, len = string.length(); i < len; i++) {
-      char ch = string.charAt(i);
-      if (ch < 0x80) {
-        sizeInBytes++; // 7-bit ASCII character
-        // This could be an ASCII run, or possibly entirely ASCII
-        while (i < len - 1) {
-          ch = string.charAt(i + 1);
-          if (ch >= 0x80) break;
-          i++;
-          sizeInBytes++; // another 7-bit ASCII character
-        }
-      } else if (ch < 0x800) {
-        sizeInBytes += 2; // 11-bit character
-      } else if (ch < 0xd800 || ch > 0xdfff) {
-        sizeInBytes += 3; // 16-bit character
-      } else {
-        int low = i + 1 < len ? string.charAt(i + 1) : 0;
-        if (ch > 0xdbff || low < 0xdc00 || low > 0xdfff) {
-          sizeInBytes++; // A malformed surrogate, which yields '?'.
-        } else {
-          // A 21-bit character
-          sizeInBytes += 4;
-          i++;
-        }
-      }
-    }
-    return sizeInBytes;
-  }
-
-  /**
-   * Binary search for character width which favors matching lower numbers.
-   *
-   * <p>Adapted from okio.Buffer
-   */
-  public static int asciiSizeInBytes(long v) {
-    if (v == 0) return 1;
-    if (v == Long.MIN_VALUE) return 20;
-
-    boolean negative = false;
-    if (v < 0) {
-      v = -v; // making this positive allows us to compare using less-than
-      negative = true;
-    }
-    int width =
-      v < 100000000L
-        ? v < 10000L
-        ? v < 100L ? v < 10L ? 1 : 2 : v < 1000L ? 3 : 4
-        : v < 1000000L ? v < 100000L ? 5 : 6 : v < 10000000L ? 7 : 8
-        : v < 1000000000000L
-          ? v < 10000000000L ? v < 1000000000L ? 9 : 10 : v < 100000000000L ? 11 : 12
-          : v < 1000000000000000L
-            ? v < 10000000000000L ? 13 : v < 100000000000000L ? 14 : 15
-            : v < 100000000000000000L
-              ? v < 10000000000000000L ? 16 : 17
-              : v < 1000000000000000000L ? 18 : 19;
-    return negative ? width + 1 : width; // conditionally add room for negative sign
-  }
-
-  /**
-   * A base 128 varint encodes 7 bits at a time, this checks how many bytes are needed to represent
-   * the value.
-   *
-   * <p>See https://developers.google.com/protocol-buffers/docs/encoding#varints
-   *
-   * <p>This logic is the same as {@code com.squareup.wire.ProtoWriter.varint32Size} v2.3.0 which
-   * benchmarked faster than loop variants of the frequently copy/pasted VarInt.varIntSize
-   */
-  public static int varintSizeInBytes(int value) {
-    if ((value & (0xffffffff << 7)) == 0) return 1;
-    if ((value & (0xffffffff << 14)) == 0) return 2;
-    if ((value & (0xffffffff << 21)) == 0) return 3;
-    if ((value & (0xffffffff << 28)) == 0) return 4;
-    return 5;
-  }
-
-  /** Like {@link #varintSizeInBytes(int)}, except for uint64. */
-  public static int varintSizeInBytes(long v) {
-    if ((v & (0xffffffffffffffffL << 7)) == 0) return 1;
-    if ((v & (0xffffffffffffffffL << 14)) == 0) return 2;
-    if ((v & (0xffffffffffffffffL << 21)) == 0) return 3;
-    if ((v & (0xffffffffffffffffL << 28)) == 0) return 4;
-    if ((v & (0xffffffffffffffffL << 35)) == 0) return 5;
-    if ((v & (0xffffffffffffffffL << 42)) == 0) return 6;
-    if ((v & (0xffffffffffffffffL << 49)) == 0) return 7;
-    if ((v & (0xffffffffffffffffL << 56)) == 0) return 8;
-    if ((v & (0xffffffffffffffffL << 63)) == 0) return 9;
-    return 10;
-  }
-
-  static void writeHexByte(byte[] data, int pos, byte b) {
-    data[pos + 0] = (byte) HEX_DIGITS[(b >> 4) & 0xf];
-    data[pos + 1] = (byte) HEX_DIGITS[b & 0xf];
-  }
-
-  public void writeAscii(String v) {
-    for (int i = 0, length = v.length(); i < length; i++) {
-      writeByte(v.charAt(i) & 0xff);
-    }
-  }
-
-  /**
-   * This transcodes a UTF-16 Java String to UTF-8 bytes.
-   *
-   * <p>This looks most similar to {@code io.netty.buffer.ByteBufUtil.writeUtf8(AbstractByteBuf,
-   * int, CharSequence, int)} v4.1, modified including features to address ASCII runs of text.
-   */
-  public void writeUtf8(CharSequence string) {
-    for (int i = 0, len = string.length(); i < len; i++) {
-      char ch = string.charAt(i);
-      if (ch < 0x80) { // 7-bit ASCII character
-        writeByte(ch);
-        // This could be an ASCII run, or possibly entirely ASCII
-        while (i < len - 1) {
-          ch = string.charAt(i + 1);
-          if (ch >= 0x80) break;
-          i++;
-          writeByte(ch); // another 7-bit ASCII character
-        }
-      } else if (ch < 0x800) { // 11-bit character
-        writeByte(0xc0 | (ch >> 6));
-        writeByte(0x80 | (ch & 0x3f));
-      } else if (ch < 0xd800 || ch > 0xdfff) { // 16-bit character
-        writeByte(0xe0 | (ch >> 12));
-        writeByte(0x80 | ((ch >> 6) & 0x3f));
-        writeByte(0x80 | (ch & 0x3f));
-      } else { // Possibly a 21-bit character
-        if (!Character.isHighSurrogate(ch)) { // Malformed or not UTF-8
-          writeByte('?');
-          continue;
-        }
-        if (i == len - 1) { // Truncated or not UTF-8
-          writeByte('?');
-          break;
-        }
-        char low = string.charAt(++i);
-        if (!Character.isLowSurrogate(low)) { // Malformed or not UTF-8
-          writeByte('?');
-          writeByte(Character.isHighSurrogate(low) ? '?' : low);
-          continue;
-        }
-        // Write the 21-bit character using 4 bytes
-        // See http://www.unicode.org/versions/Unicode7.0.0/ch03.pdf#G2630
-        int codePoint = Character.toCodePoint(ch, low);
-        writeByte(0xf0 | (codePoint >> 18));
-        writeByte(0x80 | ((codePoint >> 12) & 0x3f));
-        writeByte(0x80 | ((codePoint >> 6) & 0x3f));
-        writeByte(0x80 | (codePoint & 0x3f));
-      }
-    }
-  }
-
-  // Adapted from okio.Buffer.writeDecimalLong
-  public void writeAscii(long v) {
-    if (v == 0) {
-      require(1);
-      writeByte('0');
-      return;
-    }
-
-    if (v == Long.MIN_VALUE) {
-      writeAscii("-9223372036854775808");
-      return;
-    }
-
-    if (v < 0) {
-      writeByte('-');
-      v = -v; // needs to be positive so we can use this for an array index
-    }
-
-    writeBackwards(v);
-  }
-
-  // com.squareup.wire.ProtoWriter.writeVarint v2.3.0
-  void writeVarint(int v) {
-    while ((v & ~0x7f) != 0) {
-      writeByte((byte) ((v & 0x7f) | 0x80));
-      v >>>= 7;
-    }
-    writeByte((byte) v);
-  }
-
-  // com.squareup.wire.ProtoWriter.writeVarint v2.3.0
-  void writeVarint(long v) {
-    while ((v & ~0x7fL) != 0) {
-      writeByte((byte) ((v & 0x7f) | 0x80));
-      v >>>= 7;
-    }
-    writeByte((byte) v);
-  }
-
-  void writeLongLe(long v) {
-    writeByte((byte) (v & 0xff));
-    writeByte((byte) ((v >> 8) & 0xff));
-    writeByte((byte) ((v >> 16) & 0xff));
-    writeByte((byte) ((v >> 24) & 0xff));
-    writeByte((byte) ((v >> 32) & 0xff));
-    writeByte((byte) ((v >> 40) & 0xff));
-    writeByte((byte) ((v >> 48) & 0xff));
-    writeByte((byte) ((v >> 56) & 0xff));
-  }
-
-  long readLongLe() {
-    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() {
-    require(1);
-    return buf[pos++];
-  }
-
-  /**
-   * @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
-  int readVarint32() {
-    byte b; // negative number implies MSB set
-    if ((b = readByte()) >= 0) {
-      return b;
-    }
-    int result = b & 0x7f;
-
-    if ((b = readByte()) >= 0) {
-      return result | b << 7;
-    }
-    result |= (b & 0x7f) << 7;
-
-    if ((b = readByte()) >= 0) {
-      return result | b << 14;
-    }
-    result |= (b & 0x7f) << 14;
-
-    if ((b = readByte()) >= 0) {
-      return result | b << 21;
-    }
-    result |= (b & 0x7f) << 21;
-
-    b = readByte();
-    if ((b & 0xf0) != 0) {
-      throw new IllegalArgumentException("Greater than 32-bit varint at position " + (pos - 1));
-    }
-    return result | b << 28;
-  }
-
-  long readVarint64() {
-    byte b; // negative number implies MSB set
-    if ((b = readByte()) >= 0) {
-      return b;
-    }
-
-    long result = b & 0x7f;
-    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));
-      }
-      result |= (long) (b & 0x7f) << (i * 7);
-    }
-    return result;
-  }
-
-  public interface Writer<T> {
-    int sizeInBytes(T value);
-
-    void write(T value, UnsafeBuffer buffer);
-  }
-
-  void require(int byteCount) {
-    if (pos + byteCount > buf.length) {
-      throw new IllegalArgumentException(
-        "Truncated: length " + byteCount + " > bytes remaining " + remaining());
-    }
-  }
-}
diff --git a/zipkin/src/main/java/zipkin2/internal/V1JsonSpanReader.java b/zipkin/src/main/java/zipkin2/internal/V1JsonSpanReader.java
index 921a0b7..fff6698 100644
--- a/zipkin/src/main/java/zipkin2/internal/V1JsonSpanReader.java
+++ b/zipkin/src/main/java/zipkin2/internal/V1JsonSpanReader.java
@@ -32,10 +32,10 @@ public final class V1JsonSpanReader implements JsonReaderAdapter<V1Span> {
 
   V1Span.Builder builder;
 
-  public boolean readList(byte[] bytes, Collection<Span> out) {
-    if (bytes.length == 0) return false;
+  public boolean readList(ReadBuffer buffer, Collection<Span> out) {
+    if (buffer.available() == 0) return false;
     V1SpanConverter converter = V1SpanConverter.create();
-    JsonReader reader = new JsonReader(bytes);
+    JsonReader reader = new JsonReader(buffer);
     try {
       reader.beginArray();
       if (!reader.hasNext()) return false;
@@ -50,8 +50,7 @@ public final class V1JsonSpanReader implements JsonReaderAdapter<V1Span> {
     }
   }
 
-  @Override
-  public V1Span fromJson(JsonReader reader) throws IOException {
+  @Override public V1Span fromJson(JsonReader reader) throws IOException {
     if (builder == null) {
       builder = V1Span.newBuilder();
     } else {
@@ -123,8 +122,7 @@ public final class V1JsonSpanReader implements JsonReaderAdapter<V1Span> {
     builder.addAnnotation(timestamp, value, endpoint);
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return "Span";
   }
 
diff --git a/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java b/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java
index d16c7e1..d9c904b 100644
--- a/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java
+++ b/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java
@@ -22,18 +22,16 @@ 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 UnsafeBuffer.Writer<Span> {
+public final class V1JsonSpanWriter implements WriteBuffer.Writer<Span> {
   final V2SpanConverter converter = V2SpanConverter.create();
   final V1SpanWriter v1SpanWriter = new V1SpanWriter();
 
-  @Override
-  public int sizeInBytes(Span value) {
+  @Override public int sizeInBytes(Span value) {
     V1Span v1Span = converter.convert(value);
     return v1SpanWriter.sizeInBytes(v1Span);
   }
 
-  @Override
-  public void write(Span value, UnsafeBuffer b) {
+  @Override public void write(Span value, WriteBuffer 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 d891ca3..78365eb 100644
--- a/zipkin/src/main/java/zipkin2/internal/V1SpanWriter.java
+++ b/zipkin/src/main/java/zipkin2/internal/V1SpanWriter.java
@@ -21,18 +21,17 @@ import zipkin2.v1.V1Annotation;
 import zipkin2.v1.V1BinaryAnnotation;
 import zipkin2.v1.V1Span;
 
-import static zipkin2.internal.UnsafeBuffer.asciiSizeInBytes;
 import static zipkin2.internal.JsonEscaper.jsonEscape;
 import static zipkin2.internal.JsonEscaper.jsonEscapedSizeInBytes;
 import static zipkin2.internal.V2SpanWriter.endpointSizeInBytes;
 import static zipkin2.internal.V2SpanWriter.writeAnnotation;
+import static zipkin2.internal.WriteBuffer.asciiSizeInBytes;
 
 /** This type is only used to backport the v1 read api as it returns v1 json. */
 // @Immutable
-public final class V1SpanWriter implements UnsafeBuffer.Writer<V1Span> {
+public final class V1SpanWriter implements WriteBuffer.Writer<V1Span> {
 
-  @Override
-  public int sizeInBytes(V1Span value) {
+  @Override public int sizeInBytes(V1Span value) {
     int sizeInBytes = 29; // {"traceId":"xxxxxxxxxxxxxxxx"
     if (value.traceIdHigh() != 0L) sizeInBytes += 16;
     if (value.parentId() != 0L) {
@@ -103,8 +102,7 @@ public final class V1SpanWriter implements UnsafeBuffer.Writer<V1Span> {
     return ++sizeInBytes; // }
   }
 
-  @Override
-  public void write(V1Span value, UnsafeBuffer b) {
+  @Override public void write(V1Span value, WriteBuffer b) {
     b.writeAscii("{\"traceId\":\"");
     if (value.traceIdHigh() != 0L) b.writeLongHex(value.traceIdHigh());
     b.writeLongHex(value.traceId());
@@ -186,16 +184,15 @@ public final class V1SpanWriter implements UnsafeBuffer.Writer<V1Span> {
     b.writeByte('}');
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return "Span";
   }
 
   static byte[] legacyEndpointBytes(@Nullable Endpoint localEndpoint) {
     if (localEndpoint == null) return null;
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(endpointSizeInBytes(localEndpoint, true));
-    V2SpanWriter.writeEndpoint(localEndpoint, buffer, true);
-    return buffer.unwrap();
+    byte[] result = new byte[endpointSizeInBytes(localEndpoint, true)];
+    V2SpanWriter.writeEndpoint(localEndpoint, WriteBuffer.wrap(result, 0), true);
+    return result;
   }
 
   static int binaryAnnotationSizeInBytes(String key, String value, int endpointSize) {
@@ -209,7 +206,8 @@ public final class V1SpanWriter implements UnsafeBuffer.Writer<V1Span> {
     return sizeInBytes;
   }
 
-  static void writeBinaryAnnotation(String key, String value, @Nullable byte[] endpoint, UnsafeBuffer b) {
+  static void writeBinaryAnnotation(String key, String value, @Nullable byte[] endpoint,
+    WriteBuffer b) {
     b.writeAscii("{\"key\":\"");
     b.writeUtf8(jsonEscape(key));
     b.writeAscii("\",\"value\":\"");
diff --git a/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanReader.java b/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanReader.java
index 541e0cc..7ed27ca 100644
--- a/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanReader.java
+++ b/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanReader.java
@@ -16,14 +16,11 @@
  */
 package zipkin2.internal;
 
-import java.nio.ByteBuffer;
 import zipkin2.Endpoint;
 import zipkin2.v1.V1Span;
 
-import static zipkin2.internal.ThriftCodec.ONE;
-import static zipkin2.internal.ThriftCodec.guardLength;
+import static zipkin2.internal.ReadBuffer.ONE;
 import static zipkin2.internal.ThriftCodec.readListLength;
-import static zipkin2.internal.ThriftCodec.readUtf8;
 import static zipkin2.internal.ThriftCodec.skip;
 import static zipkin2.internal.ThriftField.TYPE_I32;
 import static zipkin2.internal.ThriftField.TYPE_I64;
@@ -48,7 +45,7 @@ public final class V1ThriftSpanReader {
 
   V1Span.Builder builder = V1Span.newBuilder();
 
-  public V1Span read(ByteBuffer bytes) {
+  public V1Span read(ReadBuffer buffer) {
     if (builder == null) {
       builder = V1Span.newBuilder();
     } else {
@@ -58,44 +55,37 @@ public final class V1ThriftSpanReader {
     ThriftField thriftField;
 
     while (true) {
-      thriftField = ThriftField.read(bytes);
+      thriftField = ThriftField.read(buffer);
       if (thriftField.type == TYPE_STOP) break;
 
       if (thriftField.isEqualTo(TRACE_ID_HIGH)) {
-        guardLength(bytes, 8);
-        builder.traceIdHigh(bytes.getLong());
+        builder.traceIdHigh(buffer.readLong());
       } else if (thriftField.isEqualTo(TRACE_ID)) {
-        guardLength(bytes, 8);
-        builder.traceId(bytes.getLong());
+        builder.traceId(buffer.readLong());
       } else if (thriftField.isEqualTo(NAME)) {
-        builder.name(readUtf8(bytes));
+        builder.name(buffer.readUtf8(buffer.readInt()));
       } else if (thriftField.isEqualTo(ID)) {
-        guardLength(bytes, 8);
-        builder.id(bytes.getLong());
+        builder.id(buffer.readLong());
       } else if (thriftField.isEqualTo(PARENT_ID)) {
-        guardLength(bytes, 8);
-        builder.parentId(bytes.getLong());
+        builder.parentId(buffer.readLong());
       } else if (thriftField.isEqualTo(ANNOTATIONS)) {
-        int length = readListLength(bytes);
+        int length = readListLength(buffer);
         for (int i = 0; i < length; i++) {
-          AnnotationReader.read(bytes, builder);
+          AnnotationReader.read(buffer, builder);
         }
       } else if (thriftField.isEqualTo(BINARY_ANNOTATIONS)) {
-        int length = readListLength(bytes);
+        int length = readListLength(buffer);
         for (int i = 0; i < length; i++) {
-          BinaryAnnotationReader.read(bytes, builder);
+          BinaryAnnotationReader.read(buffer, builder);
         }
       } else if (thriftField.isEqualTo(DEBUG)) {
-        guardLength(bytes, 1);
-        builder.debug(bytes.get() == 1);
+        builder.debug(buffer.readByte() == 1);
       } else if (thriftField.isEqualTo(TIMESTAMP)) {
-        guardLength(bytes, 8);
-        builder.timestamp(bytes.getLong());
+        builder.timestamp(buffer.readLong());
       } else if (thriftField.isEqualTo(DURATION)) {
-        guardLength(bytes, 8);
-        builder.duration(bytes.getLong());
+        builder.duration(buffer.readLong());
       } else {
-        skip(bytes, thriftField.type);
+        skip(buffer, thriftField.type);
       }
     }
 
@@ -107,25 +97,24 @@ public final class V1ThriftSpanReader {
     static final ThriftField VALUE = new ThriftField(TYPE_STRING, 2);
     static final ThriftField ENDPOINT = new ThriftField(TYPE_STRUCT, 3);
 
-    static void read(ByteBuffer bytes, V1Span.Builder builder) {
+    static void read(ReadBuffer buffer, V1Span.Builder builder) {
       long timestamp = 0;
       String value = null;
       Endpoint endpoint = null;
 
       ThriftField thriftField;
       while (true) {
-        thriftField = ThriftField.read(bytes);
+        thriftField = ThriftField.read(buffer);
         if (thriftField.type == TYPE_STOP) break;
 
         if (thriftField.isEqualTo(TIMESTAMP)) {
-          guardLength(bytes, 8);
-          timestamp = bytes.getLong();
+          timestamp = buffer.readLong();
         } else if (thriftField.isEqualTo(VALUE)) {
-          value = readUtf8(bytes);
+          value = buffer.readUtf8(buffer.readInt());
         } else if (thriftField.isEqualTo(ENDPOINT)) {
-          endpoint = ThriftEndpointCodec.read(bytes);
+          endpoint = ThriftEndpointCodec.read(buffer);
         } else {
-          skip(bytes, thriftField.type);
+          skip(buffer, thriftField.type);
         }
       }
 
@@ -140,7 +129,7 @@ public final class V1ThriftSpanReader {
     static final ThriftField TYPE = new ThriftField(TYPE_I32, 3);
     static final ThriftField ENDPOINT = new ThriftField(TYPE_STRUCT, 4);
 
-    static void read(ByteBuffer bytes, V1Span.Builder builder) {
+    static void read(ReadBuffer buffer, V1Span.Builder builder) {
       String key = null;
       String value = null;
       Endpoint endpoint = null;
@@ -148,15 +137,14 @@ public final class V1ThriftSpanReader {
       boolean isString = false;
 
       while (true) {
-        ThriftField thriftField = ThriftField.read(bytes);
+        ThriftField thriftField = ThriftField.read(buffer);
         if (thriftField.type == TYPE_STOP) break;
         if (thriftField.isEqualTo(KEY)) {
-          key = readUtf8(bytes);
+          key = buffer.readUtf8(buffer.readInt());
         } else if (thriftField.isEqualTo(VALUE)) {
-          value = readUtf8(bytes);
+          value = buffer.readUtf8(buffer.readInt());
         } else if (thriftField.isEqualTo(TYPE)) {
-          guardLength(bytes, 4);
-          switch (bytes.getInt()) {
+          switch (buffer.readInt()) {
             case 0:
               isBoolean = true;
               break;
@@ -165,9 +153,9 @@ public final class V1ThriftSpanReader {
               break;
           }
         } else if (thriftField.isEqualTo(ENDPOINT)) {
-          endpoint = ThriftEndpointCodec.read(bytes);
+          endpoint = ThriftEndpointCodec.read(buffer);
         } else {
-          skip(bytes, thriftField.type);
+          skip(buffer, thriftField.type);
         }
       }
       if (key == null || value == null) return;
@@ -181,5 +169,6 @@ public final class V1ThriftSpanReader {
     }
   }
 
-  V1ThriftSpanReader() {}
+  V1ThriftSpanReader() {
+  }
 }
diff --git a/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java b/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java
index 5c80d1c..322f727 100644
--- a/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java
+++ b/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java
@@ -24,7 +24,6 @@ import zipkin2.v1.V1BinaryAnnotation;
 import zipkin2.v1.V1Span;
 import zipkin2.v1.V2SpanConverter;
 
-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;
@@ -32,11 +31,11 @@ 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.ThriftField.TYPE_STRUCT;
+import static zipkin2.internal.WriteBuffer.utf8SizeInBytes;
 
 /** This type isn't thread-safe: it re-uses state to avoid re-allocations in conversion loops. */
 // @Immutable
-public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
-
+public final class V1ThriftSpanWriter implements WriteBuffer.Writer<Span> {
   static final ThriftField TRACE_ID = new ThriftField(TYPE_I64, 1);
   static final ThriftField TRACE_ID_HIGH = new ThriftField(TYPE_I64, 12);
   static final ThriftField NAME = new ThriftField(TYPE_STRING, 3);
@@ -52,8 +51,7 @@ public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
 
   final V2SpanConverter converter = V2SpanConverter.create();
 
-  @Override
-  public int sizeInBytes(Span value) {
+  @Override public int sizeInBytes(Span value) {
     V1Span v1Span = converter.convert(value);
 
     int endpointSize =
@@ -94,8 +92,7 @@ public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
     return sizeInBytes;
   }
 
-  @Override
-  public void write(Span value, UnsafeBuffer buffer) {
+  @Override public void write(Span value, WriteBuffer buffer) {
     V1Span v1Span = converter.convert(value);
     byte[] endpointBytes = legacyEndpointBytes(value.localEndpoint());
 
@@ -142,7 +139,7 @@ public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
     buffer.writeByte(TYPE_STOP);
   }
 
-  static void writeAnnotations(UnsafeBuffer buffer, V1Span v1Span, byte[] endpointBytes) {
+  static void writeAnnotations(WriteBuffer buffer, V1Span v1Span, byte[] endpointBytes) {
     int annotationCount = v1Span.annotations().size();
     ThriftCodec.writeListBegin(buffer, annotationCount);
     for (int i = 0; i < annotationCount; i++) {
@@ -151,7 +148,7 @@ public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
     }
   }
 
-  static void writeBinaryAnnotations(UnsafeBuffer buffer, V1Span v1Span, byte[] endpointBytes) {
+  static void writeBinaryAnnotations(WriteBuffer buffer, V1Span v1Span, byte[] endpointBytes) {
     int binaryAnnotationCount = v1Span.binaryAnnotations().size();
     ThriftCodec.writeListBegin(buffer, binaryAnnotationCount);
     for (int i = 0; i < binaryAnnotationCount; i++) {
@@ -161,8 +158,7 @@ public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
     }
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return "Span";
   }
 
@@ -170,22 +166,22 @@ public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
     int lengthOfSpans = spans.size();
     if (lengthOfSpans == 0) return EMPTY_ARRAY;
 
-    UnsafeBuffer result = UnsafeBuffer.allocate(ThriftCodec.listSizeInBytes(this, spans));
-    ThriftCodec.writeList(this, spans, result);
-    return result.unwrap();
+    byte[] result = new byte[ThriftCodec.listSizeInBytes(this, spans)];
+    ThriftCodec.writeList(this, spans, WriteBuffer.wrap(result, 0));
+    return result;
   }
 
   public byte[] write(Span onlySpan) {
-    UnsafeBuffer result = UnsafeBuffer.allocate(sizeInBytes(onlySpan));
-    write(onlySpan, result);
-    return result.unwrap();
+    byte[] result = new byte[sizeInBytes(onlySpan)];
+    write(onlySpan, WriteBuffer.wrap(result, 0));
+    return result;
   }
 
   public int writeList(List<Span> spans, byte[] out, int pos) {
     int lengthOfSpans = spans.size();
     if (lengthOfSpans == 0) return 0;
 
-    UnsafeBuffer result = UnsafeBuffer.wrap(out, pos);
+    WriteBuffer result = WriteBuffer.wrap(out, pos);
     ThriftCodec.writeList(this, spans, result);
 
     return result.pos() - pos;
@@ -193,9 +189,9 @@ public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
 
   static byte[] legacyEndpointBytes(@Nullable Endpoint localEndpoint) {
     if (localEndpoint == null) return null;
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(ThriftEndpointCodec.sizeInBytes(localEndpoint));
-    ThriftEndpointCodec.write(localEndpoint, buffer);
-    return buffer.unwrap();
+    byte[] result = new byte[ThriftEndpointCodec.sizeInBytes(localEndpoint)];
+    ThriftEndpointCodec.write(localEndpoint, WriteBuffer.wrap(result, 0));
+    return result;
   }
 
   static class ThriftAnnotationWriter {
@@ -213,7 +209,7 @@ public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
       return sizeInBytes;
     }
 
-    static void write(long timestamp, String value, byte[] endpointBytes, UnsafeBuffer buffer) {
+    static void write(long timestamp, String value, byte[] endpointBytes, WriteBuffer buffer) {
       TIMESTAMP.write(buffer);
       ThriftCodec.writeLong(buffer, timestamp);
 
@@ -245,7 +241,7 @@ public final class V1ThriftSpanWriter implements UnsafeBuffer.Writer<Span> {
       return sizeInBytes;
     }
 
-    static void write(String key, String stringValue, byte[] endpointBytes, UnsafeBuffer buffer) {
+    static void write(String key, String stringValue, byte[] endpointBytes, WriteBuffer 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 9dbba53..05ca43a 100644
--- a/zipkin/src/main/java/zipkin2/internal/V2SpanWriter.java
+++ b/zipkin/src/main/java/zipkin2/internal/V2SpanWriter.java
@@ -22,14 +22,13 @@ import zipkin2.Annotation;
 import zipkin2.Endpoint;
 import zipkin2.Span;
 
-import static zipkin2.internal.UnsafeBuffer.asciiSizeInBytes;
 import static zipkin2.internal.JsonEscaper.jsonEscape;
 import static zipkin2.internal.JsonEscaper.jsonEscapedSizeInBytes;
+import static zipkin2.internal.WriteBuffer.asciiSizeInBytes;
 
 // @Immutable
-public final class V2SpanWriter implements UnsafeBuffer.Writer<Span> {
-  @Override
-  public int sizeInBytes(Span value) {
+public final class V2SpanWriter implements WriteBuffer.Writer<Span> {
+  @Override public int sizeInBytes(Span value) {
     int sizeInBytes = 13; // {"traceId":""
     sizeInBytes += value.traceId().length();
     if (value.parentId() != null) {
@@ -88,8 +87,7 @@ public final class V2SpanWriter implements UnsafeBuffer.Writer<Span> {
     return ++sizeInBytes; // }
   }
 
-  @Override
-  public void write(Span value, UnsafeBuffer b) {
+  @Override public void write(Span value, WriteBuffer b) {
     b.writeAscii("{\"traceId\":\"");
     b.writeAscii(value.traceId());
     b.writeByte('"');
@@ -160,8 +158,7 @@ public final class V2SpanWriter implements UnsafeBuffer.Writer<Span> {
     b.writeByte('}');
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     return "Span";
   }
 
@@ -192,7 +189,7 @@ public final class V2SpanWriter implements UnsafeBuffer.Writer<Span> {
     return ++sizeInBytes; // }
   }
 
-  static void writeEndpoint(Endpoint value, UnsafeBuffer b, boolean writeEmptyServiceName) {
+  static void writeEndpoint(Endpoint value, WriteBuffer b, boolean writeEmptyServiceName) {
     b.writeByte('{');
     boolean wroteField = false;
     String serviceName = value.serviceName();
@@ -237,7 +234,8 @@ public final class V2SpanWriter implements UnsafeBuffer.Writer<Span> {
     return sizeInBytes;
   }
 
-  static void writeAnnotation(long timestamp, String value, @Nullable byte[] endpoint, UnsafeBuffer b) {
+  static void writeAnnotation(long timestamp, String value, @Nullable byte[] endpoint,
+    WriteBuffer b) {
     b.writeAscii("{\"timestamp\":");
     b.writeAscii(timestamp);
     b.writeAscii(",\"value\":\"");
diff --git a/zipkin/src/test/java/zipkin2/codec/SpanBytesDecoderTest.java b/zipkin/src/test/java/zipkin2/codec/SpanBytesDecoderTest.java
index 679c3e6..6f2ef57 100644
--- a/zipkin/src/test/java/zipkin2/codec/SpanBytesDecoderTest.java
+++ b/zipkin/src/test/java/zipkin2/codec/SpanBytesDecoderTest.java
@@ -43,7 +43,7 @@ public class SpanBytesDecoderTest {
 
   @Test public void niceErrorOnTruncatedSpans_PROTO3() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Truncated: length 66 > bytes remaining 8 reading List<Span> from proto3");
+    thrown.expectMessage("Truncated: length 66 > bytes available 8 reading List<Span> from proto3");
 
     byte[] encoded = SpanBytesEncoder.PROTO3.encodeList(TRACE);
     SpanBytesDecoder.PROTO3.decodeList(Arrays.copyOfRange(encoded, 0, 10));
@@ -51,7 +51,7 @@ public class SpanBytesDecoderTest {
 
   @Test public void niceErrorOnTruncatedSpan_PROTO3() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Truncated: length 179 > bytes remaining 7 reading Span from proto3");
+    thrown.expectMessage("Truncated: length 179 > bytes available 7 reading Span from proto3");
 
     byte[] encoded = SpanBytesEncoder.PROTO3.encode(SPAN);
     SpanBytesDecoder.PROTO3.decodeOne(Arrays.copyOfRange(encoded, 0, 10));
@@ -172,7 +172,7 @@ public class SpanBytesDecoderTest {
 
   @Test public void niceErrorOnMalformed_inputSpans_PROTO3() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Truncated: length 101 > bytes remaining 3 reading List<Span> from proto3");
+    thrown.expectMessage("Truncated: length 101 > bytes available 3 reading List<Span> from proto3");
 
     SpanBytesDecoder.PROTO3.decodeList(new byte[] {'h', 'e', 'l', 'l', 'o'});
   }
diff --git a/zipkin/src/test/java/zipkin2/codec/V1SpanBytesDecoderTest.java b/zipkin/src/test/java/zipkin2/codec/V1SpanBytesDecoderTest.java
index 4d66efa..d4af20d 100644
--- a/zipkin/src/test/java/zipkin2/codec/V1SpanBytesDecoderTest.java
+++ b/zipkin/src/test/java/zipkin2/codec/V1SpanBytesDecoderTest.java
@@ -44,7 +44,7 @@ public class V1SpanBytesDecoderTest {
 
   @Test public void niceErrorOnTruncatedSpans_THRIFT() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Truncated: length 8 > bytes remaining 2 reading List<Span> from TBinary");
+    thrown.expectMessage("Truncated: length 8 > bytes available 2 reading List<Span> from TBinary");
 
     byte[] encoded = SpanBytesEncoder.THRIFT.encodeList(TRACE);
     SpanBytesDecoder.THRIFT.decodeList(Arrays.copyOfRange(encoded, 0, 10));
@@ -52,7 +52,7 @@ public class V1SpanBytesDecoderTest {
 
   @Test public void niceErrorOnTruncatedSpan_THRIFT() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Truncated: length 8 > bytes remaining 7 reading Span from TBinary");
+    thrown.expectMessage("Truncated: length 8 > bytes available 7 reading Span from TBinary");
 
     byte[] encoded = SpanBytesEncoder.THRIFT.encode(SPAN);
     SpanBytesDecoder.THRIFT.decodeOne(Arrays.copyOfRange(encoded, 0, 10));
@@ -161,8 +161,7 @@ public class V1SpanBytesDecoderTest {
   @Test
   public void niceErrorOnMalformed_inputSpans_THRIFT() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(
-        "Truncated: length 1701604463 > bytes remaining 0 reading List<Span> from TBinary");
+    thrown.expectMessage("Truncated: length 1 > bytes available 0 reading List<Span> from TBinary");
 
     SpanBytesDecoder.THRIFT.decodeList(new byte[] {'h', 'e', 'l', 'l', 'o'});
   }
diff --git a/zipkin/src/test/java/zipkin2/internal/DependenciesTest.java b/zipkin/src/test/java/zipkin2/internal/DependenciesTest.java
index bc6c60e..7dd9c78 100644
--- a/zipkin/src/test/java/zipkin2/internal/DependenciesTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/DependenciesTest.java
@@ -24,8 +24,7 @@ import static java.util.Arrays.asList;
 import static org.assertj.core.api.Assertions.assertThat;
 
 public final class DependenciesTest {
-  @Test
-  public void dependenciesRoundTrip() {
+  @Test public void dependenciesRoundTrip() {
     DependencyLink ab = DependencyLink.newBuilder().parent("a").child("b").callCount(2L).build();
     DependencyLink cd = DependencyLink.newBuilder().parent("c").child("d").errorCount(2L).build();
 
@@ -33,7 +32,5 @@ public final class DependenciesTest {
 
     ByteBuffer bytes = dependencies.toThrift();
     assertThat(Dependencies.fromThrift(bytes)).isEqualTo(dependencies);
-
-    assertThat(bytes.remaining()).isZero();
   }
 }
diff --git a/zipkin/src/test/java/zipkin2/internal/JsonCodecTest.java b/zipkin/src/test/java/zipkin2/internal/JsonCodecTest.java
index 5421518..604033f 100644
--- a/zipkin/src/test/java/zipkin2/internal/JsonCodecTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/JsonCodecTest.java
@@ -33,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 UnsafeBuffer.Writer {
+    class FooWriter implements WriteBuffer.Writer {
       @Override public int sizeInBytes(Object value) {
         return 2;
       }
 
-      @Override public void write(Object value, UnsafeBuffer buffer) {
+      @Override public void write(Object value, WriteBuffer buffer) {
         buffer.writeByte('a');
         throw new RuntimeException("buggy");
       }
@@ -58,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 UnsafeBuffer.Writer {
+    class FooWriter implements WriteBuffer.Writer {
       @Override public int sizeInBytes(Object value) {
         return 2;
       }
 
-      @Override public void write(Object value, UnsafeBuffer buffer) {
+      @Override public void write(Object value, WriteBuffer buffer) {
         buffer.writeByte('a');
         buffer.writeByte('b');
         buffer.writeByte('c'); // wrote larger than size!
@@ -84,7 +84,7 @@ public class JsonCodecTest {
     Exception error = null;
     byte[] bytes = "[\"='".getBytes(UTF_8);
     try {
-      new JsonCodec.JsonReader(bytes).beginObject();
+      new JsonCodec.JsonReader(ReadBuffer.wrap(bytes, 0)).beginObject();
       failBecauseExceptionWasNotThrown(IllegalStateException.class);
     } catch (IOException | IllegalStateException e) {
       error = e;
diff --git a/zipkin/src/test/java/zipkin2/internal/Proto3FieldsTest.java b/zipkin/src/test/java/zipkin2/internal/Proto3FieldsTest.java
index 2893dc5..d36d644 100644
--- a/zipkin/src/test/java/zipkin2/internal/Proto3FieldsTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/Proto3FieldsTest.java
@@ -35,7 +35,8 @@ import static zipkin2.internal.Proto3Fields.WIRETYPE_LENGTH_DELIMITED;
 import static zipkin2.internal.Proto3Fields.WIRETYPE_VARINT;
 
 public class Proto3FieldsTest {
-  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
+  byte[] bytes = new byte[2048]; // bigger than needed to test sizeInBytes
+  WriteBuffer buf = WriteBuffer.wrap(bytes, 0);
 
   /** Shows we can reliably look at a byte zero to tell if we are decoding proto3 repeated fields. */
   @Test public void field_key_fieldOneLengthDelimited() {
@@ -141,24 +142,24 @@ public class Proto3FieldsTest {
     VarintField field = new VarintField(128 << 3 | WIRETYPE_VARINT);
     field.write(buf, 0xffffffffffffffffL);
 
-    buf.skip(1); // skip the key
-    skipValue(WIRETYPE_VARINT);
+    ReadBuffer readBuffer = ReadBuffer.wrap(bytes, 1 /* skip the key */);
+    skipValue(readBuffer, WIRETYPE_VARINT);
   }
 
   @Test public void field_skipValue_LENGTH_DELIMITED() {
     Utf8Field field = new Utf8Field(128 << 3 | WIRETYPE_LENGTH_DELIMITED);
     field.write(buf, "订单维护服务");
 
-    buf.skip(1); // skip the key
-    skipValue(WIRETYPE_LENGTH_DELIMITED);
+    ReadBuffer readBuffer = ReadBuffer.wrap(bytes, 1 /* skip the key */);
+    skipValue(readBuffer, WIRETYPE_LENGTH_DELIMITED);
   }
 
   @Test public void field_skipValue_FIXED64() {
     Fixed64Field field = new Fixed64Field(128 << 3 | WIRETYPE_FIXED64);
     field.write(buf, 0xffffffffffffffffL);
 
-    buf.skip(1); // skip the key
-    skipValue(WIRETYPE_FIXED64);
+    ReadBuffer readBuffer = ReadBuffer.wrap(bytes, 1 /* skip the key */);
+    skipValue(readBuffer, WIRETYPE_FIXED64);
   }
 
   @Test public void field_skipValue_FIXED32() {
@@ -169,31 +170,29 @@ public class Proto3FieldsTest {
     buf.writeByte(0xff);
     buf.writeByte(0xff);
 
-    buf.skip(1); // skip the key
-    skipValue(WIRETYPE_FIXED32);
+    ReadBuffer readBuffer = ReadBuffer.wrap(bytes, 1 /* skip the key */);
+    skipValue(readBuffer, WIRETYPE_FIXED32);
   }
 
   @Test public void field_readLengthPrefix_LENGTH_DELIMITED() {
     BytesField field = new BytesField(128 << 3 | WIRETYPE_LENGTH_DELIMITED);
     field.write(buf, new byte[10]);
-    buf.reset();
-    buf.skip(1); // skip the key
 
-    assertThat(buf.readVarint32())
+    ReadBuffer readBuffer = ReadBuffer.wrap(bytes, 1 /* skip the key */);
+    assertThat(readBuffer.readVarint32())
       .isEqualTo(10);
   }
 
   @Test public void field_readLengthPrefixAndValue_LENGTH_DELIMITED_truncated() {
     BytesField field = new BytesField(128 << 3 | WIRETYPE_LENGTH_DELIMITED);
-    buf = UnsafeBuffer.allocate(10);
-    buf.writeVarint(100); // much larger than the buffer size
-    buf.reset();
+    bytes = new byte[10];
+    WriteBuffer.wrap(bytes, 0).writeVarint(100); // much larger than the buffer size
 
     try {
-      field.readLengthPrefixAndValue(buf);
+      field.readLengthPrefixAndValue(ReadBuffer.wrap(bytes, 0));
       failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
     } catch (IllegalArgumentException e) {
-      assertThat(e).hasMessage("Truncated: length 100 > bytes remaining 9");
+      assertThat(e).hasMessage("Truncated: length 100 > bytes available 9");
     }
   }
 
@@ -201,14 +200,13 @@ public class Proto3FieldsTest {
     Fixed64Field field = new Fixed64Field(128 << 3 | WIRETYPE_FIXED64);
     field.write(buf, 0xffffffffffffffffL);
 
-    buf.reset();
-    buf.skip(1); // skip the key
-    assertThat(field.readValue(buf))
+    ReadBuffer readBuffer = ReadBuffer.wrap(bytes, 1 /* skip the key */);
+    assertThat(field.readValue(readBuffer))
       .isEqualTo(0xffffffffffffffffL);
   }
 
-  void skipValue(int wireType) {
-    assertThat(Field.skipValue(buf, wireType))
+  void skipValue(ReadBuffer buffer, int wireType) {
+    assertThat(Field.skipValue(buffer, wireType))
       .isTrue();
   }
 }
diff --git a/zipkin/src/test/java/zipkin2/internal/Proto3SpanWriterTest.java b/zipkin/src/test/java/zipkin2/internal/Proto3SpanWriterTest.java
index 03c31de..2925951 100644
--- a/zipkin/src/test/java/zipkin2/internal/Proto3SpanWriterTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/Proto3SpanWriterTest.java
@@ -24,31 +24,29 @@ import static zipkin2.TestObjects.CLIENT_SPAN;
 import static zipkin2.internal.Proto3ZipkinFields.SPAN;
 
 public class Proto3SpanWriterTest {
-  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
-
   Proto3SpanWriter writer = new Proto3SpanWriter();
 
   /** proto messages always need a key, so the non-list form is just a single-field */
   @Test public void write_startsWithSpanKeyAndLengthPrefix() {
-    byte[] buff = writer.write(CLIENT_SPAN);
+    byte[] bytes = writer.write(CLIENT_SPAN);
 
-    assertThat(buff)
+    assertThat(bytes)
       .hasSize(writer.sizeInBytes(CLIENT_SPAN))
       .startsWith((byte) 10, SPAN.sizeOfValue(CLIENT_SPAN));
   }
 
   @Test public void writeList_startsWithSpanKeyAndLengthPrefix() {
-    byte[] buff = writer.writeList(asList(CLIENT_SPAN));
+    byte[] bytes = writer.writeList(asList(CLIENT_SPAN));
 
-    assertThat(buff)
+    assertThat(bytes)
       .hasSize(writer.sizeInBytes(CLIENT_SPAN))
       .startsWith((byte) 10, SPAN.sizeOfValue(CLIENT_SPAN));
   }
 
   @Test public void writeList_multiple() {
-    byte[] buff = writer.writeList(asList(CLIENT_SPAN, CLIENT_SPAN));
+    byte[] bytes = writer.writeList(asList(CLIENT_SPAN, CLIENT_SPAN));
 
-    assertThat(buff)
+    assertThat(bytes)
       .hasSize(writer.sizeInBytes(CLIENT_SPAN) * 2)
       .startsWith((byte) 10, SPAN.sizeOfValue(CLIENT_SPAN));
   }
@@ -59,9 +57,10 @@ public class Proto3SpanWriterTest {
   }
 
   @Test public void writeList_offset_startsWithSpanKeyAndLengthPrefix() {
-    writer.writeList(asList(CLIENT_SPAN, CLIENT_SPAN), buf.unwrap(), 0);
+    byte[] bytes = new byte[2048];
+    writer.writeList(asList(CLIENT_SPAN, CLIENT_SPAN), bytes, 0);
 
-    assertThat(buf.unwrap())
+    assertThat(bytes)
       .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 ad3d139..8e4b844 100644
--- a/zipkin/src/test/java/zipkin2/internal/Proto3ZipkinFieldsTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/Proto3ZipkinFieldsTest.java
@@ -35,7 +35,8 @@ import static zipkin2.internal.Proto3Fields.WIRETYPE_LENGTH_DELIMITED;
 import static zipkin2.internal.Proto3ZipkinFields.SPAN;
 
 public class Proto3ZipkinFieldsTest {
-  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeInBytes
+  byte[] bytes = new byte[2048]; // bigger than needed to test sizeInBytes
+  WriteBuffer buf = WriteBuffer.wrap(bytes, 0);
 
   /** A map entry is an embedded messages: one for field the key and one for the value */
   @Test public void tag_sizeInBytes() {
@@ -79,14 +80,14 @@ public class Proto3ZipkinFieldsTest {
   @Test public void span_write_startsWithFieldInListOfSpans() {
     SPAN.write(buf, spanBuilder().build());
 
-    assertThat(buf.unwrap()).startsWith(
+    assertThat(bytes).startsWith(
       0b00001010 /* span key */, 20 /* bytes for length of the span */
     );
   }
 
   @Test public void span_write_writesIds() {
     SPAN.write(buf, spanBuilder().build());
-    assertThat(buf.unwrap()).startsWith(
+    assertThat(bytes).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 +155,7 @@ public class Proto3ZipkinFieldsTest {
 
   @Test public void span_write_kind() {
     SPAN.write(buf, spanBuilder().kind(Span.Kind.PRODUCER).build());
-    assertThat(buf.unwrap())
+    assertThat(bytes)
       .contains(0b0100000, atIndex(22)) // (field_number << 3) | wire_type = 4 << 3 | 0
       .contains(0b0000011, atIndex(23)); // producer's index is 3
   }
@@ -162,21 +163,19 @@ public class Proto3ZipkinFieldsTest {
   @Test public void span_read_kind_tolerant() {
     assertRoundTrip(spanBuilder().kind(Span.Kind.CONSUMER).build());
 
-    buf.reset();
-    buf.unwrap()[23] = (byte) (Span.Kind.values().length + 1); // undefined kind
-    assertThat(SPAN.read(buf))
+    bytes[23] = (byte) (Span.Kind.values().length + 1); // undefined kind
+    assertThat(SPAN.read(ReadBuffer.wrap(bytes, 0)))
       .isEqualTo(spanBuilder().build()); // skips undefined kind instead of dying
 
-    buf.reset();
-    buf.unwrap()[23] = 0; // serialized zero
-    assertThat(SPAN.read(buf))
+    bytes[23] = 0; // serialized zero
+    assertThat(SPAN.read(ReadBuffer.wrap(bytes, 0)))
       .isEqualTo(spanBuilder().build());
   }
 
   @Test public void span_write_debug() {
     SPAN.write(buf, CLIENT_SPAN.toBuilder().debug(true).build());
 
-    assertThat(buf.unwrap())
+    assertThat(bytes)
       .contains(0b01100000, atIndex(buf.pos() - 2)) // (field_number << 3) | wire_type = 12 << 3 | 0
       .contains(1, atIndex(buf.pos() - 1)); // true
   }
@@ -184,7 +183,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.unwrap())
+    assertThat(bytes)
       .contains(0b01101000, atIndex(buf.pos() - 2)) // (field_number << 3) | wire_type = 13 << 3 | 0
       .contains(1, atIndex(buf.pos() - 1)); // true
   }
@@ -195,9 +194,8 @@ public class Proto3ZipkinFieldsTest {
 
   void assertRoundTrip(Span span) {
     SPAN.write(buf, span);
-    buf.reset();
 
-    assertThat(SPAN.read(buf))
+    assertThat(SPAN.read(ReadBuffer.wrap(bytes, 0)))
       .isEqualTo(span);
   }
 }
diff --git a/zipkin/src/test/java/zipkin2/internal/UnsafeBufferTest.java b/zipkin/src/test/java/zipkin2/internal/UnsafeBufferTest.java
deleted file mode 100644
index 22f97f2..0000000
--- a/zipkin/src/test/java/zipkin2/internal/UnsafeBufferTest.java
+++ /dev/null
@@ -1,300 +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;
-
-import java.io.UnsupportedEncodingException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.util.Arrays;
-import org.junit.Test;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown;
-import static zipkin2.TestObjects.UTF_8;
-
-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++) {
-      if (codepoint == 0xD800) codepoint = 0xDFFF + 1; // skip surrogates
-      if (Character.isDefined(codepoint)) {
-        String test = new String(Character.toChars(codepoint));
-        int expected = test.getBytes(UTF_8).length;
-        int actual = UnsafeBuffer.utf8SizeInBytes(test);
-        if (actual != expected) {
-          throw new AssertionError(actual + " length != " + expected + " for " + codepoint);
-        }
-      }
-    }
-  }
-
-  /** Uses test data and codepoint wrapping trick from okhttp3.FormBodyTest */
-  @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(UnsafeBuffer.utf8SizeInBytes(test))
-        .isEqualTo(3);
-
-      UnsafeBuffer buffer = UnsafeBuffer.allocate(3);
-      buffer.writeUtf8(test);
-      assertThat(buffer.unwrap())
-        .containsExactly('a', '?', 'c');
-    }
-  }
-
-  @Test public void utf8_21Bit_truncated() {
-    // https://en.wikipedia.org/wiki/Mahjong_Tiles_(Unicode_block)
-    char[] array = "\uD83C\uDC00\uD83C\uDC01".toCharArray();
-    array[array.length - 1] = 'c';
-    String test = new String(array, 0, array.length - 1);
-    assertThat(UnsafeBuffer.utf8SizeInBytes(test))
-      .isEqualTo(5);
-
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(5);
-    buffer.writeUtf8(test);
-    assertThat(new String(buffer.unwrap(), UTF_8))
-      .isEqualTo("\uD83C\uDC00?");
-  }
-
-  @Test public void utf8_21Bit_brokenLowSurrogate() {
-    // https://en.wikipedia.org/wiki/Mahjong_Tiles_(Unicode_block)
-    char[] array = "\uD83C\uDC00\uD83C\uDC01".toCharArray();
-    array[array.length - 1] = 'c';
-    String test = new String(array);
-    assertThat(UnsafeBuffer.utf8SizeInBytes(test))
-      .isEqualTo(6);
-
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(6);
-    buffer.writeUtf8(test);
-    assertThat(new String(buffer.unwrap(), UTF_8))
-      .isEqualTo("\uD83C\uDC00?c");
-  }
-
-  @Test public void utf8_matchesJRE() {
-    // examples from http://utf8everywhere.org/
-    for (String string : Arrays.asList(
-      "Приве́т नमस्ते שָׁלוֹם",
-      "ю́ cyrillic small letter yu with acute",
-      "∃y ∀x ¬(x ≺ y)"
-    )) {
-      int encodedSize = UnsafeBuffer.utf8SizeInBytes(string);
-      assertThat(encodedSize)
-        .isEqualTo(string.getBytes(UTF_8).length);
-
-      UnsafeBuffer bufferUtf8 = UnsafeBuffer.allocate(encodedSize);
-      bufferUtf8.writeUtf8(string);
-      assertThat(new String(bufferUtf8.unwrap(), UTF_8))
-        .isEqualTo(string);
-    }
-  }
-
-  @Test public void utf8_matchesAscii() throws Exception {
-    String ascii = "86154a4ba6e913854d1e00c0db9010db";
-    int encodedSize = UnsafeBuffer.utf8SizeInBytes(ascii);
-    assertThat(encodedSize)
-      .isEqualTo(ascii.length());
-
-    UnsafeBuffer bufferAscii = UnsafeBuffer.allocate(encodedSize);
-    bufferAscii.writeAscii(ascii);
-    assertThat(new String(bufferAscii.unwrap(), "US-ASCII"))
-      .isEqualTo(ascii);
-
-    UnsafeBuffer bufferUtf8 = UnsafeBuffer.allocate(encodedSize);
-    bufferUtf8.writeUtf8(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(UnsafeBuffer.utf8SizeInBytes(emoji))
-      .isEqualTo(emojiBytes.length);
-
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(emojiBytes.length);
-    buffer.writeUtf8(emoji);
-    assertThat(buffer.unwrap())
-      .isEqualTo(emojiBytes);
-  }
-
-  @Test public void writeAscii_long() {
-    assertThat(writeAscii(-1005656679588439279L))
-      .isEqualTo("-1005656679588439279");
-    assertThat(writeAscii(0L))
-      .isEqualTo("0");
-    assertThat(writeAscii(-9223372036854775808L /* Long.MIN_VALUE */))
-      .isEqualTo("-9223372036854775808");
-    assertThat(writeAscii(123456789L))
-      .isEqualTo("123456789");
-  }
-
-  static String writeAscii(long v) {
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.asciiSizeInBytes(v));
-    buffer.writeAscii(v);
-    return new String(buffer.unwrap(), UTF_8);
-  }
-
-  // Test creating Buffer for a long string
-  @Test public void writeString() throws UnsupportedEncodingException {
-    StringBuffer stringBuffer = new StringBuffer();
-    for (int i = 0; i < 100000; i++) {
-      stringBuffer.append("a");
-    }
-    String string = stringBuffer.toString();
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(string.length());
-    buffer.writeAscii(string);
-    assertThat(new String(buffer.unwrap(), "US-ASCII")).isEqualTo(string);
-  }
-
-  @Test public void unsignedVarintSize_32_largest() {
-    // largest to encode is a negative number
-    assertThat(UnsafeBuffer.varintSizeInBytes(Integer.MIN_VALUE))
-      .isEqualTo(5);
-  }
-
-  @Test public void unsignedVarintSize_64_largest() {
-    // largest to encode is a negative number
-    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)) {
-      UnsafeBuffer buffer = UnsafeBuffer.allocate(8);
-      buffer.writeLongLe(number);
-
-      ByteBuffer byteBuffer = ByteBuffer.allocate(8);
-      byteBuffer.order(ByteOrder.LITTLE_ENDIAN);
-      byteBuffer.putLong(number);
-
-      assertThat(buffer.unwrap())
-        .containsExactly(byteBuffer.array());
-    }
-  }
-
-  // https://developers.google.com/protocol-buffers/docs/encoding#varints
-  @Test public void writeVarint_32() {
-    int number = 300;
-
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(number));
-    buffer.writeVarint(number);
-
-    assertThat(buffer.unwrap())
-      .containsExactly(0b1010_1100, 0b0000_0010);
-  }
-
-  // https://developers.google.com/protocol-buffers/docs/encoding#varints
-  @Test public void writeVarint_64() {
-    long number = 300;
-
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(number));
-    buffer.writeVarint(number);
-
-    assertThat(buffer.unwrap())
-      .containsExactly(0b1010_1100, 0b0000_0010);
-  }
-
-  @Test public void writeVarint_ports() {
-    // normal case
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(80));
-    buffer.writeVarint(80);
-
-    assertThat(buffer.unwrap())
-      .containsExactly(0b0101_0000);
-
-    // largest value to not require more than 2 bytes (14 bits set)
-    buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(16383));
-    buffer.writeVarint(16383);
-
-    assertThat(buffer.unwrap())
-      .containsExactly(0b1111_1111, 0b0111_1111);
-
-    // worst case is a byte longer than fixed 16
-    buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(65535));
-    buffer.writeVarint(65535);
-
-    assertThat(buffer.unwrap())
-      .containsExactly(0b1111_1111, 0b1111_1111, 0b0000_0011);
-
-    // most bits
-    buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(0xFFFFFFFF));
-    buffer.writeVarint(0xFFFFFFFF);
-
-    // we have a total of 32 bits encoded
-    assertThat(buffer.unwrap())
-      .containsExactly(0b1111_1111, 0b1111_1111, 0b1111_1111, 0b1111_1111, 0b0000_1111);
-  }
-
-  @Test public void readVarint32() {
-    assertReadVarint32(0);
-    assertReadVarint32(0b0011_1111_1111_1111);
-    assertReadVarint32(0xFFFFFFFF);
-  }
-
-  static void assertReadVarint32(int value) {
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(value));
-    buffer.writeVarint(value);
-    buffer.reset();
-
-    assertThat(buffer.readVarint32())
-      .isEqualTo(value);
-  }
-
-  @Test public void readVarint32_malformedTooBig() {
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(8);
-    buffer.writeLongLe(0xffffffffffffL);
-    buffer.reset();
-
-    try {
-      buffer.readVarint32();
-      failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
-    } catch (IllegalArgumentException e) {
-      assertThat(e)
-        .hasMessage("Greater than 32-bit varint at position 4");
-    }
-  }
-
-  @Test public void readVarint64() {
-    assertReadVarint64(0L);
-    assertReadVarint64(0b0011_1111_1111_1111L);
-    assertReadVarint64(0xffffffffffffffffL);
-  }
-
-  static void assertReadVarint64(long value) {
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(value));
-    buffer.writeVarint(value);
-    buffer.reset();
-
-    assertThat(buffer.readVarint64())
-      .isEqualTo(value);
-  }
-
-  @Test public void readVarint64_malformedTooBig() {
-    UnsafeBuffer buffer = UnsafeBuffer.allocate(16);
-    buffer.writeLongLe(0xffffffffffffffffL);
-    buffer.writeLongLe(0xffffffffffffffffL);
-    buffer.reset();
-
-    try {
-      buffer.readVarint64();
-      failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
-    } catch (IllegalArgumentException e) {
-      assertThat(e)
-        .hasMessage("Greater than 64-bit varint at position 9");
-    }
-  }
-}
diff --git a/zipkin/src/test/java/zipkin2/internal/V1JsonSpanWriterTest.java b/zipkin/src/test/java/zipkin2/internal/V1JsonSpanWriterTest.java
index 7f6ae0a..468b759 100644
--- a/zipkin/src/test/java/zipkin2/internal/V1JsonSpanWriterTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/V1JsonSpanWriterTest.java
@@ -16,18 +16,18 @@
  */
 package zipkin2.internal;
 
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
 import org.junit.Test;
 import zipkin2.Endpoint;
 import zipkin2.Span;
 
 import static org.assertj.core.api.Assertions.assertThat;
 import static zipkin2.TestObjects.CLIENT_SPAN;
+import static zipkin2.internal.JsonCodec.UTF_8;
 
 public class V1JsonSpanWriterTest {
   V1JsonSpanWriter writer = new V1JsonSpanWriter();
-  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
+  byte[] bytes = new byte[2048]; // bigger than needed to test sizeInBytes
+  WriteBuffer buf = WriteBuffer.wrap(bytes, 0);
 
   @Test
   public void sizeInBytes() {
@@ -37,153 +37,149 @@ public class V1JsonSpanWriterTest {
   }
 
   @Test
-  public void writesCoreAnnotations_client() throws IOException {
+  public void writesCoreAnnotations_client() {
     writer.write(CLIENT_SPAN, buf);
 
     writesCoreAnnotations("cs", "cr");
   }
 
   @Test
-  public void writesCoreAnnotations_server() throws IOException {
+  public void writesCoreAnnotations_server() {
     writer.write(CLIENT_SPAN.toBuilder().kind(Span.Kind.SERVER).build(), buf);
 
     writesCoreAnnotations("sr", "ss");
   }
 
   @Test
-  public void writesCoreAnnotations_producer() throws IOException {
+  public void writesCoreAnnotations_producer() {
     writer.write(CLIENT_SPAN.toBuilder().kind(Span.Kind.PRODUCER).build(), buf);
 
     writesCoreAnnotations("ms", "ws");
   }
 
   @Test
-  public void writesCoreAnnotations_consumer() throws IOException {
+  public void writesCoreAnnotations_consumer() {
     writer.write(CLIENT_SPAN.toBuilder().kind(Span.Kind.CONSUMER).build(), buf);
 
     writesCoreAnnotations("wr", "mr");
   }
 
-  void writesCoreAnnotations(String begin, String end) throws UnsupportedEncodingException {
-    String json = new String(buf.unwrap(), "UTF-8");
+  void writesCoreAnnotations(String begin, String end) {
+    String json = new String(bytes, UTF_8);
 
     assertThat(json)
         .contains("{\"timestamp\":" + CLIENT_SPAN.timestamp() + ",\"value\":\"" + begin + "\"");
     assertThat(json)
-        .contains(
-            "{\"timestamp\":"
-                + (CLIENT_SPAN.timestamp() + CLIENT_SPAN.duration())
-                + ",\"value\":\""
-                + end
-                + "\"");
+        .contains("{\"timestamp\":"
+          + (CLIENT_SPAN.timestampAsLong() + CLIENT_SPAN.durationAsLong())
+          + ",\"value\":\"" + end + "\"");
   }
 
   @Test
-  public void writesCoreSendAnnotations_client() throws IOException {
+  public void writesCoreSendAnnotations_client() {
     writer.write(CLIENT_SPAN.toBuilder().duration(null).build(), buf);
 
     writesCoreSendAnnotations("cs");
   }
 
   @Test
-  public void writesCoreSendAnnotations_server() throws IOException {
+  public void writesCoreSendAnnotations_server() {
     writer.write(CLIENT_SPAN.toBuilder().duration(null).kind(Span.Kind.SERVER).build(), buf);
 
     writesCoreSendAnnotations("sr");
   }
 
   @Test
-  public void writesCoreSendAnnotations_producer() throws IOException {
+  public void writesCoreSendAnnotations_producer() {
     writer.write(CLIENT_SPAN.toBuilder().duration(null).kind(Span.Kind.PRODUCER).build(), buf);
 
     writesCoreSendAnnotations("ms");
   }
 
   @Test
-  public void writesCoreSendAnnotations_consumer() throws IOException {
+  public void writesCoreSendAnnotations_consumer() {
     writer.write(CLIENT_SPAN.toBuilder().duration(null).kind(Span.Kind.CONSUMER).build(), buf);
 
     writesCoreSendAnnotations("mr");
   }
 
-  void writesCoreSendAnnotations(String begin) throws UnsupportedEncodingException {
-    String json = new String(buf.unwrap(), "UTF-8");
+  void writesCoreSendAnnotations(String begin) {
+    String json = new String(bytes, UTF_8);
 
     assertThat(json)
         .contains("{\"timestamp\":" + CLIENT_SPAN.timestamp() + ",\"value\":\"" + begin + "\"");
   }
 
   @Test
-  public void writesAddressBinaryAnnotation_client() throws IOException {
+  public void writesAddressBinaryAnnotation_client() {
     writer.write(CLIENT_SPAN.toBuilder().build(), buf);
 
     writesAddressBinaryAnnotation("sa");
   }
 
   @Test
-  public void writesAddressBinaryAnnotation_server() throws IOException {
+  public void writesAddressBinaryAnnotation_server() {
     writer.write(CLIENT_SPAN.toBuilder().kind(Span.Kind.SERVER).build(), buf);
 
     writesAddressBinaryAnnotation("ca");
   }
 
   @Test
-  public void writesAddressBinaryAnnotation_producer() throws IOException {
+  public void writesAddressBinaryAnnotation_producer() {
     writer.write(CLIENT_SPAN.toBuilder().kind(Span.Kind.PRODUCER).build(), buf);
 
     writesAddressBinaryAnnotation("ma");
   }
 
   @Test
-  public void writesAddressBinaryAnnotation_consumer() throws IOException {
+  public void writesAddressBinaryAnnotation_consumer() {
     writer.write(CLIENT_SPAN.toBuilder().kind(Span.Kind.CONSUMER).build(), buf);
 
     writesAddressBinaryAnnotation("ma");
   }
 
-  void writesAddressBinaryAnnotation(String address) throws UnsupportedEncodingException {
-    String json = new String(buf.unwrap(), "UTF-8");
-
-    assertThat(json).contains("{\"key\":\"" + address + "\",\"value\":true,\"endpoint\":");
+  void writesAddressBinaryAnnotation(String address) {
+    assertThat(new String(bytes, UTF_8))
+      .contains("{\"key\":\"" + address + "\",\"value\":true,\"endpoint\":");
   }
 
   @Test
-  public void writes128BitTraceId() throws UnsupportedEncodingException {
+  public void writes128BitTraceId() {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
         .startsWith("{\"traceId\":\"" + CLIENT_SPAN.traceId() + "\"");
   }
 
   @Test
-  public void annotationsHaveEndpoints() throws IOException {
+  public void annotationsHaveEndpoints() {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
         .contains(
             "\"value\":\"foo\",\"endpoint\":{\"serviceName\":\"frontend\",\"ipv4\":\"127.0.0.1\"}");
   }
 
   @Test
-  public void writesTimestampAndDuration() throws IOException {
+  public void writesTimestampAndDuration() {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
         .contains(
             "\"timestamp\":" + CLIENT_SPAN.timestamp() + ",\"duration\":" + CLIENT_SPAN.duration());
   }
 
   @Test
-  public void skipsTimestampAndDuration_shared() throws IOException {
+  public void skipsTimestampAndDuration_shared() {
     writer.write(CLIENT_SPAN.toBuilder().kind(Span.Kind.SERVER).shared(true).build(), buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
         .doesNotContain(
             "\"timestamp\":" + CLIENT_SPAN.timestamp() + ",\"duration\":" + CLIENT_SPAN.duration());
   }
 
   @Test
-  public void writesEmptySpanName() throws IOException {
+  public void writesEmptySpanName() {
     Span span =
         Span.newBuilder()
             .traceId("7180c278b62e8f6a216a2aea45d08fc9")
@@ -193,11 +189,11 @@ public class V1JsonSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8")).contains("\"name\":\"\"");
+    assertThat(new String(bytes, UTF_8)).contains("\"name\":\"\"");
   }
 
   @Test
-  public void writesEmptyServiceName() throws IOException {
+  public void writesEmptyServiceName() {
     Span span =
         CLIENT_SPAN
             .toBuilder()
@@ -206,15 +202,15 @@ public class V1JsonSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
         .contains("\"value\":\"foo\",\"endpoint\":{\"serviceName\":\"\",\"ipv4\":\"127.0.0.1\"}");
   }
 
   @Test
-  public void tagsAreBinaryAnnotations() throws IOException {
+  public void tagsAreBinaryAnnotations() {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, 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 ee32150..c5b3aed 100644
--- a/zipkin/src/test/java/zipkin2/internal/V1ThriftSpanWriterTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/V1ThriftSpanWriterTest.java
@@ -16,7 +16,6 @@
  */
 package zipkin2.internal;
 
-import java.nio.ByteBuffer;
 import org.junit.Before;
 import org.junit.Test;
 import zipkin2.Endpoint;
@@ -38,32 +37,28 @@ 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();
-  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
+  byte[] bytes = new byte[2048]; // bigger than needed to test sizeOf
+  WriteBuffer buf = WriteBuffer.wrap(bytes, 0);
 
   V1ThriftSpanWriter writer = new V1ThriftSpanWriter();
-  byte[] endpointBytes;
+  byte[] endpointBytes = new byte[ThriftEndpointCodec.sizeInBytes(endpoint)];
 
-  @Before
-  public void init() {
-    UnsafeBuffer endpointBuffer = UnsafeBuffer.allocate(ThriftEndpointCodec.sizeInBytes(endpoint));
-    ThriftEndpointCodec.write(endpoint, endpointBuffer);
-    endpointBytes = endpointBuffer.unwrap();
+  @Before public void init() {
+    ThriftEndpointCodec.write(endpoint, WriteBuffer.wrap(endpointBytes, 0));
   }
 
-
   @Test
   public void endpoint_highPort() {
     int highPort = 63840;
     Endpoint endpoint = Endpoint.newBuilder().ip("127.0.0.1").port(63840).build();
-    UnsafeBuffer endpointBuffer = UnsafeBuffer.allocate(ThriftEndpointCodec.sizeInBytes(endpoint));
-    ThriftEndpointCodec.write(endpoint, endpointBuffer);
-    byte[] buff = endpointBuffer.unwrap();
+    byte[] buff = new byte[ThriftEndpointCodec.sizeInBytes(endpoint)];
+    ThriftEndpointCodec.write(endpoint, WriteBuffer.wrap(buff, 0));
 
     assertThat(buff)
       .containsSequence(TYPE_I32, 0, 1, 127, 0, 0, 1) // ipv4
       .containsSequence(TYPE_I16, 0, 2, (highPort >> 8) & 0xFF, highPort & 0xFF); // port
 
-    assertThat(ThriftEndpointCodec.read(ByteBuffer.wrap(buff)).portAsInt())
+    assertThat(ThriftEndpointCodec.read(ReadBuffer.wrap(buff, 0)).portAsInt())
       .isEqualTo(highPort);
   }
 
@@ -72,8 +67,8 @@ public class V1ThriftSpanWriterTest {
     byte[] buff = writer.write(span);
 
     assertThat(buff)
-        .hasSize(writer.sizeInBytes(span))
-        .startsWith(TYPE_I64, 0, 1); // short value of field number 1
+      .hasSize(writer.sizeInBytes(span))
+      .startsWith(TYPE_I64, 0, 1); // short value of field number 1
   }
 
   @Test
@@ -81,9 +76,9 @@ public class V1ThriftSpanWriterTest {
     byte[] buff = writer.writeList(asList(span));
 
     assertThat(buff)
-        .hasSize(5 + writer.sizeInBytes(span))
-        .startsWith( // member type of the list and an integer with the count
-            TYPE_STRUCT, 0, 0, 0, 1);
+      .hasSize(5 + writer.sizeInBytes(span))
+      .startsWith( // member type of the list and an integer with the count
+        TYPE_STRUCT, 0, 0, 0, 1);
   }
 
   @Test
@@ -91,9 +86,9 @@ public class V1ThriftSpanWriterTest {
     byte[] buff = writer.writeList(asList(span, span));
 
     assertThat(buff)
-        .hasSize(5 + writer.sizeInBytes(span) * 2)
-        .startsWith( // member type of the list and an integer with the count
-            TYPE_STRUCT, 0, 0, 0, 2);
+      .hasSize(5 + writer.sizeInBytes(span) * 2)
+      .startsWith( // member type of the list and an integer with the count
+        TYPE_STRUCT, 0, 0, 0, 2);
   }
 
   @Test
@@ -103,21 +98,21 @@ public class V1ThriftSpanWriterTest {
 
   @Test
   public void writeList_offset_startsWithListPrefix() {
-    writer.writeList(asList(span, span), buf.unwrap(), 1);
+    writer.writeList(asList(span, span), bytes, 1);
 
-    assertThat(buf.unwrap())
-        .startsWith( // member type of the list and an integer with the count
-            0, TYPE_STRUCT, 0, 0, 0, 2);
+    assertThat(bytes)
+      .startsWith( // member type of the list and an integer with the count
+        0, TYPE_STRUCT, 0, 0, 0, 2);
   }
 
   @Test
   public void doesntWriteAnnotationsWhenMissingTimestamp() {
     writer.write(span.toBuilder().kind(CLIENT).build(), buf);
 
-    UnsafeBuffer buf2 = UnsafeBuffer.allocate(2048);
-    writer.write(span, buf2);
+    byte[] bytes2 = new byte[2048];
+    writer.write(span, WriteBuffer.wrap(bytes2, 0));
 
-    assertThat(buf.unwrap()).containsExactly(buf.unwrap());
+    assertThat(bytes).containsExactly(bytes2);
   }
 
   @Test
@@ -144,12 +139,12 @@ public class V1ThriftSpanWriterTest {
     span = span.toBuilder().kind(kind).timestamp(5).duration(10).build();
     writer.write(span, buf);
 
-    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))
-        .containsSequence(TYPE_I64, 0, 1, 0, 0, 0, 0, 0, 0, 0, 15) // timestamp
-        .containsSequence(TYPE_STRING, 0, 2, 0, 0, 0, 2, end.charAt(0), end.charAt(1));
+    assertThat(bytes)
+      .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))
+      .containsSequence(TYPE_I64, 0, 1, 0, 0, 0, 0, 0, 0, 0, 15) // timestamp
+      .containsSequence(TYPE_STRING, 0, 2, 0, 0, 0, 2, end.charAt(0), end.charAt(1));
   }
 
   @Test
@@ -176,10 +171,10 @@ public class V1ThriftSpanWriterTest {
     span = span.toBuilder().kind(kind).timestamp(5).build();
     writer.write(span, buf);
 
-    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));
+    assertThat(bytes)
+      .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));
   }
 
   @Test
@@ -205,42 +200,32 @@ public class V1ThriftSpanWriterTest {
   void writesAddressBinaryAnnotation(Span.Kind kind, String addr) {
     writer.write(span.toBuilder().kind(kind).remoteEndpoint(endpoint).build(), buf);
 
-    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
-        .containsSequence(TYPE_I32, 0, 3, 0, 0, 0, 0) // type 0 == boolean
-        .containsSequence(endpointBytes);
+    assertThat(bytes)
+      .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
+      .containsSequence(TYPE_I32, 0, 3, 0, 0, 0, 0) // type 0 == boolean
+      .containsSequence(endpointBytes);
   }
 
   @Test
   public void annotationsHaveEndpoints() {
     writer.write(span.toBuilder().localEndpoint(endpoint).addAnnotation(5, "foo").build(), buf);
 
-    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
-        .containsSequence(endpointBytes);
+    assertThat(bytes)
+      .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
+      .containsSequence(endpointBytes);
   }
 
   @Test
   public void writesTimestampAndDuration() {
     writer.write(span.toBuilder().timestamp(5).duration(10).build(), buf);
 
-    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
-  }
-
-  @Test
-  public void skipsTimestampAndDuration_shared() {
-    writer.write(span.toBuilder().kind(SERVER).timestamp(5).duration(10).shared(true).build(), buf);
-
-    UnsafeBuffer buf2 = UnsafeBuffer.allocate(2048);
-    writer.write(span.toBuilder().kind(SERVER).build(), buf2);
-
-    assertThat(buf.unwrap()).containsExactly(buf.unwrap());
+    assertThat(bytes)
+      .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
   }
 
   @Test
@@ -249,30 +234,30 @@ public class V1ThriftSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(buf.unwrap())
-        .containsSequence(
-            ThriftField.TYPE_STRING, 0, 3, 0, 0, 0, 0); // name (empty is 32 zero bits)
+    assertThat(bytes)
+      .containsSequence(
+        ThriftField.TYPE_STRING, 0, 3, 0, 0, 0, 0); // name (empty is 32 zero bits)
   }
 
   @Test
   public void writesTraceAndSpanIds() {
     writer.write(span, buf);
 
-    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
+    assertThat(bytes)
+      .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
   }
 
   @Test
   public void writesParentAnd128BitTraceId() {
     writer.write(
-        Span.newBuilder().traceId("00000000000000010000000000000002").parentId("3").id("4").build(),
-        buf);
+      Span.newBuilder().traceId("00000000000000010000000000000002").parentId("3").id("4").build(),
+      buf);
 
-    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
+    assertThat(bytes)
+      .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
   }
 
   /** For finagle compatibility */
@@ -282,9 +267,9 @@ public class V1ThriftSpanWriterTest {
 
     writer.write(span, buf);
 
-    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
+    assertThat(bytes)
+      .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
   }
 
   @Test
@@ -293,27 +278,27 @@ public class V1ThriftSpanWriterTest {
 
     writer.write(span, buf);
 
-    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
-        .containsSequence(TYPE_I32, 0, 3, 0, 0, 0, 6) // type 6 == string
-        .containsSequence(endpointBytes);
+    assertThat(bytes)
+      .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
+      .containsSequence(TYPE_I32, 0, 3, 0, 0, 0, 6) // type 6 == string
+      .containsSequence(endpointBytes);
   }
 
   @Test
   public void writesEmptyServiceName() {
     span =
-        span.toBuilder()
-            .name("foo")
-            .localEndpoint(Endpoint.newBuilder().ip("127.0.0.1").build())
-            .build();
+      span.toBuilder()
+        .name("foo")
+        .localEndpoint(Endpoint.newBuilder().ip("127.0.0.1").build())
+        .build();
 
     writer.write(span, buf);
 
-    assertThat(buf.unwrap())
-        .containsSequence(
-            ThriftField.TYPE_STRING, 0, 3, 0, 0, 0, 0); // serviceName (empty is 32 zero bits)
+    assertThat(bytes)
+      .containsSequence(
+        ThriftField.TYPE_STRING, 0, 3, 0, 0, 0, 0); // serviceName (empty is 32 zero bits)
   }
 
   /** To match finagle */
@@ -323,17 +308,17 @@ public class V1ThriftSpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(buf.unwrap()).containsSequence(ThriftField.TYPE_BOOL, 0);
+    assertThat(bytes).containsSequence(ThriftField.TYPE_BOOL, 0);
   }
 
   @Test
   public void tagsAreBinaryAnnotations() {
     writer.write(span.toBuilder().putTag("foo", "bar").build(), buf);
 
-    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
-        .containsSequence(TYPE_I32, 0, 3, 0, 0, 0, 6); // type 6 == string
+    assertThat(bytes)
+      .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
+      .containsSequence(TYPE_I32, 0, 3, 0, 0, 0, 6); // type 6 == string
   }
 }
diff --git a/zipkin/src/test/java/zipkin2/internal/V2SpanWriterTest.java b/zipkin/src/test/java/zipkin2/internal/V2SpanWriterTest.java
index 8c632ed..8f33835 100644
--- a/zipkin/src/test/java/zipkin2/internal/V2SpanWriterTest.java
+++ b/zipkin/src/test/java/zipkin2/internal/V2SpanWriterTest.java
@@ -16,8 +16,6 @@
  */
 package zipkin2.internal;
 
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -27,10 +25,12 @@ import zipkin2.Span;
 import static org.assertj.core.api.Assertions.assertThat;
 import static zipkin2.TestObjects.CLIENT_SPAN;
 import static zipkin2.TestObjects.TODAY;
+import static zipkin2.TestObjects.UTF_8;
 
 public class V2SpanWriterTest {
   V2SpanWriter writer = new V2SpanWriter();
-  UnsafeBuffer buf = UnsafeBuffer.allocate(2048); // bigger than needed to test sizeOf
+  byte[] bytes = new byte[2048]; // bigger than needed to test sizeInBytes
+  WriteBuffer buf = WriteBuffer.wrap(bytes, 0);
 
   @Rule public ExpectedException thrown = ExpectedException.none();
 
@@ -40,21 +40,21 @@ public class V2SpanWriterTest {
       .isEqualTo(buf.pos());
   }
 
-  @Test public void writes128BitTraceId() throws UnsupportedEncodingException {
+  @Test public void writes128BitTraceId() {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
       .startsWith("{\"traceId\":\"" + CLIENT_SPAN.traceId() + "\"");
   }
 
-  @Test public void writesAnnotationWithoutEndpoint() throws IOException {
+  @Test public void writesAnnotationWithoutEndpoint() {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
       .contains("{\"timestamp\":" + (TODAY + 100) * 1000L + ",\"value\":\"foo\"}");
   }
 
-  @Test public void omitsEmptySpanName() throws IOException {
+  @Test public void omitsEmptySpanName() {
     Span span = Span.newBuilder()
       .traceId("7180c278b62e8f6a216a2aea45d08fc9")
       .parentId("6b221d5bc9e6496c")
@@ -63,25 +63,25 @@ public class V2SpanWriterTest {
 
     writer.write(span, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
       .doesNotContain("name");
   }
 
-  @Test public void omitsEmptyServiceName() throws IOException {
+  @Test public void omitsEmptyServiceName() {
     Span span = CLIENT_SPAN.toBuilder()
       .localEndpoint(Endpoint.newBuilder().ip("127.0.0.1").build())
       .build();
 
     writer.write(span, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
       .contains("\"localEndpoint\":{\"ipv4\":\"127.0.0.1\"}");
   }
 
-  @Test public void tagsAreAMap() throws IOException {
+  @Test public void tagsAreAMap() {
     writer.write(CLIENT_SPAN, buf);
 
-    assertThat(new String(buf.unwrap(), "UTF-8"))
+    assertThat(new String(bytes, UTF_8))
       .contains("\"tags\":{\"clnt/finagle.version\":\"6.45.0\",\"http.path\":\"/api\"}");
   }
 }