From 24fc499a7015e8e9f3b3571f23b100f9106881f9 Mon Sep 17 00:00:00 2001 From: Adrian Cole Date: Wed, 15 May 2019 14:21:45 +0800 Subject: [PATCH] Makes SpanBytesDecoder work on ByteBuffer (#2589) * 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 * whoops forgot the important parts * de-optimizes for thrift in order to increase clarity * bug * Adds json comparison benchmarks * just review feedback * mah endian * less code * bench --- benchmarks/pom.xml | 7 + .../zipkin2/codec/JsonCodecBenchmarks.java | 104 +++++ .../java/zipkin2/codec/MoshiSpanDecoder.java | 95 +++++ .../zipkin2/codec/ProtoCodecBenchmarks.java | 4 +- .../elasticsearch/ElasticsearchAccess.java | 26 ++ .../internal/ReadBufferBenchmarks.java | 126 ++++++ ...hmarks.java => WriteBufferBenchmarks.java} | 22 +- .../zipkin2/codec/MoshiSpanDecoderTest.java | 40 ++ .../internal/Proto3CodecInteropTest.java | 34 +- .../server/internal/ZipkinQueryApiV2.java | 8 +- .../storage/cassandra/v1/CassandraUtil.java | 1 - .../cassandra/v1/SelectFromTraces.java | 25 +- zipkin/src/main/java/zipkin2/Endpoint.java | 2 +- zipkin/src/main/java/zipkin2/Span.java | 2 +- .../codec/DependencyLinkBytesDecoder.java | 75 ++-- .../codec/DependencyLinkBytesEncoder.java | 9 +- .../java/zipkin2/codec/SpanBytesDecoder.java | 167 +++++--- .../java/zipkin2/internal/Dependencies.java | 47 +-- .../main/java/zipkin2/internal/HexCodec.java | 4 + .../main/java/zipkin2/internal/JsonCodec.java | 69 ++- .../java/zipkin2/internal/JsonEscaper.java | 2 +- .../java/zipkin2/internal/Proto3Codec.java | 14 +- .../java/zipkin2/internal/Proto3Fields.java | 53 +-- .../zipkin2/internal/Proto3SpanWriter.java | 21 +- .../zipkin2/internal/Proto3ZipkinFields.java | 24 +- .../java/zipkin2/internal/ReadBuffer.java | 395 ++++++++++++++++++ .../java/zipkin2/internal/ThriftCodec.java | 136 ++---- .../zipkin2/internal/ThriftEndpointCodec.java | 22 +- .../java/zipkin2/internal/ThriftField.java | 10 +- .../zipkin2/internal/V1JsonSpanReader.java | 12 +- .../zipkin2/internal/V1JsonSpanWriter.java | 8 +- .../java/zipkin2/internal/V1SpanWriter.java | 22 +- .../zipkin2/internal/V1ThriftSpanReader.java | 74 ++-- .../zipkin2/internal/V1ThriftSpanWriter.java | 42 +- .../java/zipkin2/internal/V2SpanWriter.java | 18 +- .../{UnsafeBuffer.java => WriteBuffer.java} | 373 +++++------------ .../zipkin2/codec/SpanBytesDecoderTest.java | 6 +- .../zipkin2/codec/V1SpanBytesDecoderTest.java | 7 +- .../zipkin2/internal/DependenciesTest.java | 5 +- .../java/zipkin2/internal/JsonCodecTest.java | 10 +- .../zipkin2/internal/Proto3FieldsTest.java | 42 +- .../internal/Proto3SpanWriterTest.java | 19 +- .../internal/Proto3ZipkinFieldsTest.java | 26 +- .../java/zipkin2/internal/ReadBufferTest.java | 239 +++++++++++ .../internal/V1JsonSpanWriterTest.java | 82 ++-- .../internal/V1ThriftSpanWriterTest.java | 177 ++++---- .../zipkin2/internal/V2SpanWriterTest.java | 26 +- ...feBufferTest.java => WriteBufferTest.java} | 186 +++------ 48 files changed, 1833 insertions(+), 1085 deletions(-) create mode 100644 benchmarks/src/main/java/zipkin2/codec/JsonCodecBenchmarks.java create mode 100644 benchmarks/src/main/java/zipkin2/codec/MoshiSpanDecoder.java create mode 100644 benchmarks/src/main/java/zipkin2/elasticsearch/ElasticsearchAccess.java create mode 100644 benchmarks/src/main/java/zipkin2/internal/ReadBufferBenchmarks.java rename benchmarks/src/main/java/zipkin2/internal/{UnsafeBufferBenchmarks.java => WriteBufferBenchmarks.java} (85%) create mode 100644 benchmarks/src/test/java/zipkin2/codec/MoshiSpanDecoderTest.java create mode 100644 zipkin/src/main/java/zipkin2/internal/ReadBuffer.java rename zipkin/src/main/java/zipkin2/internal/{UnsafeBuffer.java => WriteBuffer.java} (65%) create mode 100644 zipkin/src/test/java/zipkin2/internal/ReadBufferTest.java rename zipkin/src/test/java/zipkin2/internal/{UnsafeBufferTest.java => WriteBufferTest.java} (52%) diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index b4b79f9933e..04174f234e2 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -94,6 +94,13 @@ org.apache.zipkin.proto3 zipkin-proto3 + + + ${project.groupId}.zipkin2 + zipkin-tests + ${project.version} + test + diff --git a/benchmarks/src/main/java/zipkin2/codec/JsonCodecBenchmarks.java b/benchmarks/src/main/java/zipkin2/codec/JsonCodecBenchmarks.java new file mode 100644 index 00000000000..abe5cbd0b0d --- /dev/null +++ b/benchmarks/src/main/java/zipkin2/codec/JsonCodecBenchmarks.java @@ -0,0 +1,104 @@ +/* + * 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.codec; + +import com.google.common.io.Resources; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +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.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; +import zipkin2.Span; + +@Measurement(iterations = 5, time = 1) +@Warmup(iterations = 10, time = 1) +@Fork(3) +@BenchmarkMode(Mode.SampleTime) +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@State(Scope.Thread) +@Threads(1) +public class JsonCodecBenchmarks { + static final MoshiSpanDecoder MOSHI = MoshiSpanDecoder.create(); + + static final byte[] clientSpanJsonV2 = read("/zipkin2-client.json"); + static final Span clientSpan = SpanBytesDecoder.JSON_V2.decodeOne(clientSpanJsonV2); + + // Assume a message is 1000 spans (which is a high number for as this is per-node-second) + static final List spans = Collections.nCopies(1000, clientSpan); + static final byte[] encodedBytes = SpanBytesEncoder.JSON_V2.encodeList(spans); + + private ByteBuf encodedBuf; + + @Setup public void setup() { + encodedBuf = PooledByteBufAllocator.DEFAULT.buffer(encodedBytes.length); + encodedBuf.writeBytes(encodedBytes); + } + + @TearDown public void tearDown() { + encodedBuf.release(); + } + + @Benchmark public List bytes_moshiDecoder() { + return MOSHI.decodeList(encodedBytes); + } + + @Benchmark public List bytes_zipkinDecoder() { + return SpanBytesDecoder.JSON_V2.decodeList(encodedBytes); + } + + @Benchmark public List bytebuffer_moshiDecoder() { + return MOSHI.decodeList(encodedBuf.nioBuffer()); + } + + @Benchmark public List bytebuffer_zipkinDecoder() { + return SpanBytesDecoder.JSON_V2.decodeList(encodedBuf.nioBuffer()); + } + + // Convenience main entry-point + public static void main(String[] args) throws Exception { + Options opt = new OptionsBuilder() + .include(".*" + JsonCodecBenchmarks.class.getSimpleName()) + .addProfiler("gc") + .build(); + + new Runner(opt).run(); + } + + static byte[] read(String resource) { + try { + return Resources.toByteArray(Resources.getResource(CodecBenchmarks.class, resource)); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } +} diff --git a/benchmarks/src/main/java/zipkin2/codec/MoshiSpanDecoder.java b/benchmarks/src/main/java/zipkin2/codec/MoshiSpanDecoder.java new file mode 100644 index 00000000000..73839b3e9b2 --- /dev/null +++ b/benchmarks/src/main/java/zipkin2/codec/MoshiSpanDecoder.java @@ -0,0 +1,95 @@ +/* + * 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.codec; + +import com.squareup.moshi.JsonAdapter; +import com.squareup.moshi.JsonReader; +import com.squareup.moshi.Moshi; +import com.squareup.moshi.Types; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import okio.Buffer; +import okio.BufferedSource; +import okio.Okio; +import okio.Timeout; +import zipkin2.Span; +import zipkin2.elasticsearch.ElasticsearchAccess; + +/** + * Read-only json adapters resurrected from before we switched to Java 6 as storage components can + * be Java 7+ + */ +public final class MoshiSpanDecoder { + final JsonAdapter> listSpansAdapter; + + public static MoshiSpanDecoder create() { + return new MoshiSpanDecoder(); + } + + MoshiSpanDecoder() { + listSpansAdapter = new Moshi.Builder() + .add(Span.class, ElasticsearchAccess.jsonSpanAdapter()) + .build().adapter(Types.newParameterizedType(List.class, Span.class)); + } + + public List decodeList(byte[] spans) { + BufferedSource source = new Buffer().write(spans); + try { + return listSpansAdapter.fromJson(source); + } catch (IOException e) { + throw new AssertionError(e); // no I/O + } + } + + public List decodeList(ByteBuffer spans) { + try { + return listSpansAdapter.fromJson(JsonReader.of(Okio.buffer(new ByteBufferSource(spans)))); + } catch (IOException e) { + throw new AssertionError(e); // no I/O + } + } + + final class ByteBufferSource implements okio.Source { + final ByteBuffer source; + + final Buffer.UnsafeCursor cursor = new Buffer.UnsafeCursor(); + + ByteBufferSource(ByteBuffer source) { + this.source = source; + } + + @Override public long read(Buffer sink, long byteCount) { + try (Buffer.UnsafeCursor ignored = sink.readAndWriteUnsafe(cursor)) { + long oldSize = sink.size(); + int length = (int) Math.min(source.remaining(), Math.min(8192, byteCount)); + if (length == 0) return -1; + cursor.expandBuffer(length); + source.get(cursor.data, cursor.start, length); + cursor.resizeBuffer(oldSize + length); + return length; + } + } + + @Override public Timeout timeout() { + return Timeout.NONE; + } + + @Override public void close() { + } + } +} diff --git a/benchmarks/src/main/java/zipkin2/codec/ProtoCodecBenchmarks.java b/benchmarks/src/main/java/zipkin2/codec/ProtoCodecBenchmarks.java index a8631587b68..67e257623d7 100644 --- a/benchmarks/src/main/java/zipkin2/codec/ProtoCodecBenchmarks.java +++ b/benchmarks/src/main/java/zipkin2/codec/ProtoCodecBenchmarks.java @@ -87,7 +87,7 @@ public List bytes_wireDecoder() { @Benchmark public List bytebuffer_zipkinDecoder() { - return SpanBytesDecoder.PROTO3.decodeList(ByteBufUtil.getBytes(encodedBuf)); + return SpanBytesDecoder.PROTO3.decodeList(encodedBuf.nioBuffer()); } @Benchmark @@ -103,7 +103,7 @@ public List bytebuffer_wireDecoder() { // Convenience main entry-point public static void main(String[] args) throws Exception { Options opt = new OptionsBuilder() - .include(".*" + ProtoCodecBenchmarks.class.getSimpleName() + ".*bytes.*") + .include(".*" + ProtoCodecBenchmarks.class.getSimpleName()) .addProfiler("gc") .build(); diff --git a/benchmarks/src/main/java/zipkin2/elasticsearch/ElasticsearchAccess.java b/benchmarks/src/main/java/zipkin2/elasticsearch/ElasticsearchAccess.java new file mode 100644 index 00000000000..ab19d2fd3b2 --- /dev/null +++ b/benchmarks/src/main/java/zipkin2/elasticsearch/ElasticsearchAccess.java @@ -0,0 +1,26 @@ +/* + * 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.elasticsearch; + +import com.squareup.moshi.JsonAdapter; +import zipkin2.Span; + +public class ElasticsearchAccess { + public static JsonAdapter jsonSpanAdapter() { + return JsonAdapters.SPAN_ADAPTER; + } +} diff --git a/benchmarks/src/main/java/zipkin2/internal/ReadBufferBenchmarks.java b/benchmarks/src/main/java/zipkin2/internal/ReadBufferBenchmarks.java new file mode 100644 index 00000000000..35fc7d21284 --- /dev/null +++ b/benchmarks/src/main/java/zipkin2/internal/ReadBufferBenchmarks.java @@ -0,0 +1,126 @@ +/* + * 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.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.SampleTime) +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@State(Scope.Thread) +@Threads(1) +public class ReadBufferBenchmarks { + byte[] longBuff = { + (byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04, + (byte) 0x05, (byte) 0x06, (byte) 0x07, (byte) 0x08, + }; + + @Benchmark public long readLong() { + int pos = 0; + return (longBuff[pos] & 0xffL) << 56 + | (longBuff[pos + 1] & 0xffL) << 48 + | (longBuff[pos + 2] & 0xffL) << 40 + | (longBuff[pos + 3] & 0xffL) << 32 + | (longBuff[pos + 4] & 0xffL) << 24 + | (longBuff[pos + 5] & 0xffL) << 16 + | (longBuff[pos + 6] & 0xffL) << 8 + | (longBuff[pos + 7] & 0xffL); + } + + @Benchmark public long readLong_localArray() { + int pos = 0; + byte[] longBuff = this.longBuff; + return (longBuff[pos] & 0xffL) << 56 + | (longBuff[pos + 1] & 0xffL) << 48 + | (longBuff[pos + 2] & 0xffL) << 40 + | (longBuff[pos + 3] & 0xffL) << 32 + | (longBuff[pos + 4] & 0xffL) << 24 + | (longBuff[pos + 5] & 0xffL) << 16 + | (longBuff[pos + 6] & 0xffL) << 8 + | (longBuff[pos + 7] & 0xffL); + } + + @Benchmark public long readLong_8arity_localArray() { + int pos = 0; + return readLong( + longBuff[pos] & 0xff, + longBuff[pos + 1] & 0xff, + longBuff[pos + 2] & 0xff, + longBuff[pos + 3] & 0xff, + longBuff[pos + 4] & 0xff, + longBuff[pos + 5] & 0xff, + longBuff[pos + 6] & 0xff, + longBuff[pos + 7] & 0xff + ); + } + + @Benchmark public long readLong_8arity() { + int pos = 0; + byte[] longBuff = this.longBuff; + return readLong( + longBuff[pos] & 0xff, + longBuff[pos + 1] & 0xff, + longBuff[pos + 2] & 0xff, + longBuff[pos + 3] & 0xff, + longBuff[pos + 4] & 0xff, + longBuff[pos + 5] & 0xff, + longBuff[pos + 6] & 0xff, + longBuff[pos + 7] & 0xff + ); + } + + static long readLong(int p0, int p1, int p2, int p3, int p4, int p5, int p6, int p7) { + return (p0 & 0xffL) << 56 + | (p1 & 0xffL) << 48 + | (p2 & 0xffL) << 40 + | (p3 & 0xffL) << 32 + | (p4 & 0xffL) << 24 + | (p5 & 0xffL) << 16 + | (p6 & 0xffL) << 8 + | (p7 & 0xffL); + } + + @Benchmark public long readLongReverseBytes() { + return Long.reverseBytes(readLong()); + } + + // Convenience main entry-point + public static void main(String[] args) throws RunnerException { + Options opt = new OptionsBuilder() + .include(".*" + ReadBufferBenchmarks.class.getSimpleName() + ".*") + .addProfiler("gc") + .build(); + + new Runner(opt).run(); + } +} diff --git a/benchmarks/src/main/java/zipkin2/internal/UnsafeBufferBenchmarks.java b/benchmarks/src/main/java/zipkin2/internal/WriteBufferBenchmarks.java similarity index 85% rename from benchmarks/src/main/java/zipkin2/internal/UnsafeBufferBenchmarks.java rename to benchmarks/src/main/java/zipkin2/internal/WriteBufferBenchmarks.java index dac80603962..28e10ae2bbd 100644 --- a/benchmarks/src/main/java/zipkin2/internal/UnsafeBufferBenchmarks.java +++ b/benchmarks/src/main/java/zipkin2/internal/WriteBufferBenchmarks.java @@ -41,7 +41,7 @@ @OutputTimeUnit(TimeUnit.MICROSECONDS) @State(Scope.Thread) @Threads(1) -public class UnsafeBufferBenchmarks { +public class WriteBufferBenchmarks { static final Charset UTF_8 = Charset.forName("UTF-8"); // Order id = d07c4daa-0fa9-4c03-90b1-e06c4edae250 doesn't exist static final String CHINESE_UTF8 = "订单d07c4daa-0fa9-4c03-90b1-e06c4edae250不存在"; @@ -50,16 +50,17 @@ public class UnsafeBufferBenchmarks { static final int TEST_INT = 1024; /* epoch micros timestamp */ static final long TEST_LONG = 1472470996199000L; - UnsafeBuffer buffer = UnsafeBuffer.allocate(8); + byte[] bytes = new byte[8]; + WriteBuffer buffer = WriteBuffer.wrap(bytes); @Benchmark public int utf8SizeInBytes_chinese() { - return UnsafeBuffer.utf8SizeInBytes(CHINESE_UTF8); + return WriteBuffer.utf8SizeInBytes(CHINESE_UTF8); } @Benchmark public byte[] writeUtf8_chinese() { - UnsafeBuffer bufferUtf8 = UnsafeBuffer.allocate(CHINESE_UTF8_SIZE); - bufferUtf8.writeUtf8(CHINESE_UTF8); - return bufferUtf8.unwrap(); + byte[] bytesUtf8 = new byte[CHINESE_UTF8_SIZE]; + WriteBuffer.wrap(bytesUtf8, 0).writeUtf8(CHINESE_UTF8); + return bytesUtf8; } @Benchmark public ByteBuffer writeUtf8_chinese_jdk() { @@ -67,27 +68,24 @@ public class UnsafeBufferBenchmarks { } @Benchmark public int varIntSizeInBytes_32() { - return UnsafeBuffer.varintSizeInBytes(TEST_INT); + return WriteBuffer.varintSizeInBytes(TEST_INT); } @Benchmark public int varIntSizeInBytes_64() { - return UnsafeBuffer.varintSizeInBytes(TEST_LONG); + return WriteBuffer.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(); } @@ -95,7 +93,7 @@ public class UnsafeBufferBenchmarks { // Convenience main entry-point public static void main(String[] args) throws RunnerException { Options opt = new OptionsBuilder() - .include(".*" + UnsafeBufferBenchmarks.class.getSimpleName() + ".*") + .include(".*" + WriteBufferBenchmarks.class.getSimpleName() + ".*") .build(); new Runner(opt).run(); diff --git a/benchmarks/src/test/java/zipkin2/codec/MoshiSpanDecoderTest.java b/benchmarks/src/test/java/zipkin2/codec/MoshiSpanDecoderTest.java new file mode 100644 index 00000000000..7e61f8ebc63 --- /dev/null +++ b/benchmarks/src/test/java/zipkin2/codec/MoshiSpanDecoderTest.java @@ -0,0 +1,40 @@ +/* + * 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.codec; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; +import org.junit.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static zipkin2.TestObjects.TRACE; + +public class MoshiSpanDecoderTest { + byte[] encoded = SpanBytesEncoder.JSON_V2.encodeList(TRACE); + + @Test public void decodeList_bytes() { + assertThat(new MoshiSpanDecoder().decodeList(encoded)) + .isEqualTo(TRACE); + } + + @Test public void decodeList_byteBuffer() { + ByteBuf encodedBuf = PooledByteBufAllocator.DEFAULT.buffer(encoded.length); + encodedBuf.writeBytes(encoded); + assertThat(new MoshiSpanDecoder().decodeList(encoded)) + .isEqualTo(TRACE); + } +} diff --git a/benchmarks/src/test/java/zipkin2/internal/Proto3CodecInteropTest.java b/benchmarks/src/test/java/zipkin2/internal/Proto3CodecInteropTest.java index d4755387228..64d50038bc5 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()); 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 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 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 265a89bea21..4427f794b4d 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.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 @@ static AggregatedHttpMessage jsonResponse(byte[] body) { .setInt(HttpHeaderNames.CONTENT_LENGTH, body.length).build(), HttpData.of(body)); } - static final UnsafeBuffer.Writer QUOTED_STRING_WRITER = new UnsafeBuffer.Writer() { + static final WriteBuffer.Writer QUOTED_STRING_WRITER = new WriteBuffer.Writer() { @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 4510e71c278..39e268c3fd3 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.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 f9636f2064a..1064578b8bb 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.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 @@ static class Factory { 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 @@ static class Factory { Call> newCall(String hexTraceId) { long traceId = HexCodec.lowerHexToUnsignedLong(hexTraceId); Call> 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 @@ FlatMapper, List>> newFlatMapper(QueryRequest request) { @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 @@ public Call>> map(Set input) { traceIds = input; } Call>> 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 @@ protected BiConsumer> accumulator() { 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 9684541b269..2b7ddbe2d6f 100644 --- a/zipkin/src/main/java/zipkin2/Endpoint.java +++ b/zipkin/src/main/java/zipkin2/Endpoint.java @@ -27,7 +27,7 @@ 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 8d22bce5e9b..7334a11648a 100644 --- a/zipkin/src/main/java/zipkin2/Span.java +++ b/zipkin/src/main/java/zipkin2/Span.java @@ -36,7 +36,7 @@ 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 90334c02ae0..b3fc0dd4fca 100644 --- a/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesDecoder.java +++ b/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesDecoder.java @@ -24,66 +24,59 @@ 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 { JSON_V1 { - @Override - public Encoding encoding() { + @Override public Encoding encoding() { return Encoding.JSON; } - @Override - public boolean decode(byte[] link, Collection out) { - return JsonCodec.read(READER, link, out); + @Override public boolean decode(byte[] link, Collection out) { + return JsonCodec.read(READER, ReadBuffer.wrap(link), 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)); } - @Override - public boolean decodeList(byte[] links, Collection out) { - return JsonCodec.readList(READER, links, out); + @Override public boolean decodeList(byte[] links, Collection out) { + return JsonCodec.readList(READER, ReadBuffer.wrap(links), out); } - @Override - public List decodeList(byte[] links) { + @Override public List decodeList(byte[] links) { List out = new ArrayList<>(); if (!decodeList(links, out)) return Collections.emptyList(); return out; } }; - static final JsonCodec.JsonReaderAdapter READER = - new JsonCodec.JsonReaderAdapter() { - @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 READER = new JsonReaderAdapter() { + @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 aee13b51b6a..12973e47dec 100644 --- a/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java +++ b/zipkin/src/main/java/zipkin2/codec/DependencyLinkBytesEncoder.java @@ -19,11 +19,12 @@ 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 { JSON_V1 { @@ -44,7 +45,7 @@ public enum DependencyLinkBytesEncoder implements BytesEncoder { } }; - static final UnsafeBuffer.Writer WRITER = new UnsafeBuffer.Writer() { + static final Writer WRITER = new Writer() { @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 { 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 7c3e290a944..1239be12195 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.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 @@ public enum SpanBytesDecoder implements BytesDecoder { /** 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 out) { - Span result = decodeOne(bytes); + @Override public boolean decode(byte[] span, Collection out) { // ex DependencyLinker + Span result = decodeOne(ReadBuffer.wrap(span)); if (result == null) return false; out.add(result); return true; } - @Override - public boolean decodeList(byte[] spans, Collection out) { - return new V1JsonSpanReader().readList(spans, out); + @Override public boolean decodeList(byte[] spans, Collection out) { // ex getTrace + return new V1JsonSpanReader().readList(ReadBuffer.wrap(spans), out); } - @Override - public Span decodeOne(byte[] span) { - V1Span v1 = JsonCodec.readOne(new V1JsonSpanReader(), span); + @Override public boolean decodeList(ByteBuffer spans, Collection out) { + return new V1JsonSpanReader().readList(ReadBuffer.wrapUnsafe(spans), out); + } + + @Override @Nullable public Span decodeOne(byte[] span) { + return decodeOne(ReadBuffer.wrap(span)); + } + + @Override @Nullable public Span decodeOne(ByteBuffer span) { + return decodeOne(ReadBuffer.wrapUnsafe(span)); + } + + Span decodeOne(ReadBuffer buffer) { + V1Span v1 = JsonCodec.readOne(new V1JsonSpanReader(), buffer); List out = new ArrayList<>(1); V1SpanConverter.create().convert(v1, out); return out.get(0); } - @Override - public List decodeList(byte[] spans) { + @Override public List decodeList(byte[] spans) { + return decodeList(this, spans); + } + + @Override public List 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 out) { - return ThriftCodec.read(span, out); + @Override public boolean decode(byte[] span, Collection out) { // ex DependencyLinker + return ThriftCodec.read(ReadBuffer.wrap(span), out); } - @Override - public boolean decodeList(byte[] spans, Collection out) { - return ThriftCodec.readList(spans, out); + @Override public boolean decodeList(byte[] spans, Collection out) { // ex getTrace + return ThriftCodec.readList(ReadBuffer.wrap(spans), out); } - @Override - public Span decodeOne(byte[] span) { - return ThriftCodec.readOne(span); + @Override public boolean decodeList(ByteBuffer spans, Collection out) { + return ThriftCodec.readList(ReadBuffer.wrapUnsafe(spans), out); } - @Override - public List decodeList(byte[] spans) { + @Override @Nullable public Span decodeOne(byte[] span) { + return ThriftCodec.readOne(ReadBuffer.wrap(span)); + } + + @Override @Nullable public Span decodeOne(ByteBuffer span) { + return ThriftCodec.readOne(ReadBuffer.wrapUnsafe(span)); + } + + @Override public List decodeList(byte[] spans) { + return decodeList(this, spans); + } + + @Override public List 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 out) { // ex decode span in dependencies job - return JsonCodec.read(new V2SpanReader(), span, out); + @Override public boolean decode(byte[] span, Collection out) { // ex DependencyLinker + return JsonCodec.read(new V2SpanReader(), ReadBuffer.wrap(span), out); + } + + @Override public boolean decodeList(byte[] spans, Collection out) { // ex getTrace + return JsonCodec.readList(new V2SpanReader(), ReadBuffer.wrap(spans), out); } - @Override - public boolean decodeList(byte[] spans, Collection out) { // ex getTrace - return JsonCodec.readList(new V2SpanReader(), spans, out); + @Override public boolean decodeList(ByteBuffer spans, Collection 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)); } - @Override - public List decodeList(byte[] spans) { + @Override @Nullable public Span decodeOne(ByteBuffer span) { + return JsonCodec.readOne(new V2SpanReader(), ReadBuffer.wrapUnsafe(span)); + } + + @Override public List decodeList(byte[] spans) { + return decodeList(this, spans); + } + + @Override public List 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 out) { // ex decode span in dependencies job - return Proto3Codec.read(span, out); + @Override public boolean decode(byte[] span, Collection out) { // ex DependencyLinker + return Proto3Codec.read(ReadBuffer.wrap(span), out); + } + + @Override public boolean decodeList(byte[] spans, Collection out) { // ex getTrace + return Proto3Codec.readList(ReadBuffer.wrap(spans), out); + } + + @Override public boolean decodeList(ByteBuffer spans, Collection out) { + return Proto3Codec.readList(ReadBuffer.wrapUnsafe(spans), out); } - @Override - public boolean decodeList(byte[] spans, Collection out) { // ex getTrace - return Proto3Codec.readList(spans, out); + @Override @Nullable public Span decodeOne(byte[] span) { + return Proto3Codec.readOne(ReadBuffer.wrap(span)); } - @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 decodeList(byte[] spans) { + @Override public List decodeList(byte[] spans) { + return decodeList(this, spans); + } + + @Override public List decodeList(ByteBuffer spans) { return decodeList(this, spans); } }; + /** + * ByteBuffer implementation of {@link #decodeList(byte[])}. + * + *

Note: only use this when it is ok to modify the underlying {@link ByteBuffer#array()}. + */ + public abstract boolean decodeList(ByteBuffer spans, Collection out); + + /** + * ByteBuffer implementation of {@link #decodeList(byte[])}. + * + *

Note: only use this when it is ok to modify the underlying {@link ByteBuffer#array()}. + */ + public abstract List decodeList(ByteBuffer spans); + + /** + * ByteBuffer implementation of {@link #decodeOne(byte[])} + * + *

Note: only use this when it is ok to modify the underlying {@link ByteBuffer#array()}. + */ + @Nullable public abstract Span decodeOne(ByteBuffer span); + static List decodeList(SpanBytesDecoder decoder, byte[] spans) { List out = new ArrayList<>(); if (!decoder.decodeList(spans, out)) return Collections.emptyList(); return out; } + + static List decodeList(SpanBytesDecoder decoder, ByteBuffer spans) { + List 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 6141ed519b1..adf5c937be4 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_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 List links() { 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 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 static Dependencies fromThrift(ByteBuffer bytes) { /** 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)); + return ByteBuffer.wrap(result); } int sizeInBytes() { @@ -90,7 +91,7 @@ int sizeInBytes() { return sizeInBytes; } - void write(UnsafeBuffer buffer) { + void write(WriteBuffer buffer) { START_TS.write(buffer); ThriftCodec.writeLong(buffer, startTs); @@ -138,39 +139,38 @@ public int hashCode() { return h; } - static final class DependencyLinkAdapter implements UnsafeBuffer.Writer { + static final class DependencyLinkAdapter implements WriteBuffer.Writer { 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 int sizeInBytes(DependencyLink value) { 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 e90e0a10b98..0739b3be872 100644 --- a/zipkin/src/main/java/zipkin2/internal/HexCodec.java +++ b/zipkin/src/main/java/zipkin2/internal/HexCodec.java @@ -18,6 +18,10 @@ // 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 0ecf4cc0df3..7787d0e6fa4 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 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. * *

    *
  • Eliminates the need to keep separate model classes for proto3 vs json @@ -44,14 +44,14 @@ * 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 boolean peekNull() throws IOException { 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 fromJson(JsonReader reader) throws IOException; } - public static boolean read(JsonReaderAdapter adapter, byte[] bytes, Collection out) { - if (bytes.length == 0) return false; + public static boolean read( + JsonReaderAdapter adapter, ReadBuffer buffer, Collection 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 readOne(JsonReaderAdapter adapter, byte[] bytes) { + public static @Nullable T readOne(JsonReaderAdapter adapter, ReadBuffer buffer) { List 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 boolean readList( - JsonReaderAdapter adapter, byte[] bytes, Collection out) { - if (bytes.length == 0) return false; - JsonReader reader = new JsonReader(bytes); + JsonReaderAdapter adapter, ReadBuffer buffer, Collection 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 static boolean readList( } } - static int sizeInBytes(UnsafeBuffer.Writer writer, List value) { + static int sizeInBytes(WriteBuffer.Writer writer, List value) { int length = value.size(); int sizeInBytes = 2; // [] if (length > 1) sizeInBytes += length - 1; // comma to join elements @@ -168,15 +166,15 @@ static int sizeInBytes(UnsafeBuffer.Writer writer, List value) { } /** Inability to encode is a programming bug. */ - public static byte[] write(UnsafeBuffer.Writer writer, T value) { - UnsafeBuffer b = UnsafeBuffer.allocate(writer.sizeInBytes(value)); + public static byte[] write(WriteBuffer.Writer writer, T value) { + byte[] result = new byte[writer.sizeInBytes(value)]; + WriteBuffer b = WriteBuffer.wrap(result); 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 static byte[] write(UnsafeBuffer.Writer writer, T value) { 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 byte[] writeList(UnsafeBuffer.Writer writer, List value) { + public static byte[] writeList(WriteBuffer.Writer writer, List 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)); + return result; } - public static int writeList(UnsafeBuffer.Writer writer, List value, byte[] out, int pos) { + public static int writeList(WriteBuffer.Writer writer, List 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 void writeList(UnsafeBuffer.Writer writer, List value, UnsafeBuffer b) { + public static void writeList(WriteBuffer.Writer writer, List 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 77f00930cde..f28cc7be44f 100644 --- a/zipkin/src/main/java/zipkin2/internal/JsonEscaper.java +++ b/zipkin/src/main/java/zipkin2/internal/JsonEscaper.java @@ -99,6 +99,6 @@ public static int jsonEscapedSizeInBytes(CharSequence v) { } } 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 52d646e70dd..add62edd157 100644 --- a/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java +++ b/zipkin/src/main/java/zipkin2/internal/Proto3Codec.java @@ -44,9 +44,8 @@ public int writeList(List spans, byte[] out, int pos) { return writer.writeList(spans, out, pos); } - public static boolean read(byte[] bytes, Collection out) { - if (bytes.length == 0) return false; - UnsafeBuffer buffer = UnsafeBuffer.wrap(bytes, 0); + public static boolean read(ReadBuffer buffer, Collection out) { + if (buffer.available() == 0) return false; try { Span span = SPAN.read(buffer); if (span == null) return false; @@ -57,18 +56,17 @@ public static boolean read(byte[] bytes, Collection out) { } } - 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 out) { - int length = bytes.length; + public static boolean readList(ReadBuffer buffer, Collection 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 b08ce902927..ab199ed82ef 100644 --- a/zipkin/src/main/java/zipkin2/internal/Proto3Fields.java +++ b/zipkin/src/main/java/zipkin2/internal/Proto3Fields.java @@ -18,6 +18,9 @@ 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 @@ static int wireType(int key, int byteL) { 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 @@ static boolean skipValue(UnsafeBuffer buffer, int wireType) { } 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 int sizeInBytes(T value) { 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 void write(UnsafeBuffer b, T value) { * 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 T readLengthPrefixAndValue(UnsafeBuffer b) { 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 { @@ -146,11 +149,11 @@ static class BytesField extends LengthDelimitedField { 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 @@ static class HexField extends LengthDelimitedField { 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 @@ static int decodeLowerHex(char c) { 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 @@ static class Utf8Field extends LengthDelimitedField { } @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 @@ static final class Fixed64Field extends Field { 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 @@ int sizeInBytes(long number) { return 1 + 8; // tag + 8 byte number } - long readValue(UnsafeBuffer buffer) { + long readValue(ReadBuffer buffer) { return buffer.readLongLe(); } } @@ -232,20 +235,20 @@ static class VarintField extends Field { } 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 @@ int sizeInBytes(boolean bool) { 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 @@ int sizeInBytes(int number) { } 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 8e2d8cd7ba4..fc5947463b2 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.Proto3ZipkinFields.SPAN; //@Immutable -final class Proto3SpanWriter implements UnsafeBuffer.Writer { +final class Proto3SpanWriter implements WriteBuffer.Writer { static final byte[] EMPTY_ARRAY = new byte[0]; @@ -31,7 +31,7 @@ final class Proto3SpanWriter implements UnsafeBuffer.Writer { 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 @@ public byte[] writeList(List spans) { 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); 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)); + 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 @@ int writeList(List 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); 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 5d273efb079..a5c94ea2864 100644 --- a/zipkin/src/main/java/zipkin2/internal/Proto3ZipkinFields.java +++ b/zipkin/src/main/java/zipkin2/internal/Proto3ZipkinFields.java @@ -69,14 +69,14 @@ static class EndpointField extends LengthDelimitedField { 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 @@ static abstract class SpanBuilderField extends LengthDelimitedField { 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 { @@ -133,12 +133,12 @@ static class AnnotationField extends SpanBuilderField { 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 @@ static final class TagField extends SpanBuilderField> return KEY.sizeInBytes(value.getKey()) + VALUE.sizeInBytes(value.getValue()); } - @Override void writeValue(UnsafeBuffer b, Map.Entry value) { + @Override void writeValue(WriteBuffer b, Map.Entry 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 @@ static class SpanField extends LengthDelimitedField { 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 @@ int toByte(Span.Kind kind) { 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 @@ public Span read(UnsafeBuffer buffer) { } } - 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/ReadBuffer.java b/zipkin/src/main/java/zipkin2/internal/ReadBuffer.java new file mode 100644 index 00000000000..d9eea1db24a --- /dev/null +++ b/zipkin/src/main/java/zipkin2/internal/ReadBuffer.java @@ -0,0 +1,395 @@ +/* + * 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.InputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import static zipkin2.internal.HexCodec.HEX_DIGITS; +import static zipkin2.internal.JsonCodec.UTF_8; + +/** Read operations do bounds checks, as typically more errors occur reading than writing. */ +public abstract class ReadBuffer extends InputStream { + + /** Do not use the buffer passed here after, as it may be manipulated directly. */ + public static ReadBuffer wrapUnsafe(ByteBuffer buffer) { + if (buffer.hasArray()) { + int offset = buffer.arrayOffset() + buffer.position(); + int limit = offset + buffer.remaining(); + return wrap(buffer.array(), offset, limit); + } + return buffer.order() == ByteOrder.BIG_ENDIAN + ? new BigEndianByteBuffer(buffer) + : new LittleEndianByteBuffer(buffer); + } + + public static ReadBuffer wrap(byte[] bytes) { + return wrap(bytes, 0, bytes.length); + } + + public static ReadBuffer wrap(byte[] bytes, int pos, int limit) { + return new ReadBuffer.Array(bytes, pos, limit); + } + + static final class BigEndianByteBuffer extends Buff { + BigEndianByteBuffer(ByteBuffer buf) { + super(buf); + } + + @Override short readShort() { + require(2); + return buf.getShort(); + } + + @Override int readInt() { + require(4); + return buf.getInt(); + } + + @Override long readLong() { + require(8); + return buf.getLong(); + } + + @Override long readLongLe() { + return Long.reverseBytes(readLong()); + } + } + + static final class LittleEndianByteBuffer extends Buff { + LittleEndianByteBuffer(ByteBuffer buf) { + super(buf); + } + + @Override short readShort() { + require(2); + return Short.reverseBytes(buf.getShort()); + } + + @Override int readInt() { + require(4); + return Integer.reverseBytes(buf.getInt()); + } + + @Override long readLong() { + return Long.reverseBytes(readLongLe()); + } + + @Override long readLongLe() { + require(8); + return buf.getLong(); + } + } + + static abstract class Buff extends ReadBuffer { + final ByteBuffer buf; // visible for testing + + Buff(ByteBuffer buf) { + this.buf = buf; + } + + @Override final byte readByteUnsafe() { + return buf.get(); + } + + @Override final byte[] readBytes(int length) { + require(length); + byte[] copy = new byte[length]; + buf.get(copy); + return copy; + } + + @Override boolean tryReadAscii(char[] destination, int length) { + buf.mark(); + for (int i = 0; i < length; i++) { + byte b = buf.get(); + if ((b & 0x80) != 0) { + buf.reset(); + return false; // Not 7-bit ASCII character + } + destination[i] = (char) b; + } + return true; + } + + @Override final String doReadUtf8(int length) { + return new String(readBytes(length), UTF_8); + } + + @Override public int pos() { + return buf.position(); + } + + @Override public int read(byte[] dst, int offset, int length) { + int toRead = checkReadArguments(dst, offset, length); + if (toRead == 0) return 0; + buf.get(dst, offset, toRead); + return toRead; + } + + @Override public long skip(long maxCount) { + int skipped = Math.max(available(), (int) maxCount); + buf.position(buf.position() + skipped); + return skipped; + } + + @Override public int available() { + return buf.remaining(); + } + } + + static final class Array extends ReadBuffer { + final byte[] buf; + int offset, length; + + Array(byte[] buf, int offset, int length) { + this.buf = buf; + this.offset = offset; + this.length = length; + } + + @Override final byte readByteUnsafe() { + return buf[offset++]; + } + + @Override final byte[] readBytes(int length) { + require(length); + byte[] result = new byte[length]; + System.arraycopy(buf, offset, result, 0, length); + offset += length; + return result; + } + + @Override public int read(byte[] dst, int offset, int length) { + int toRead = checkReadArguments(dst, offset, length); + if (toRead == 0) return 0; + System.arraycopy(buf, this.offset, dst, 0, toRead); + this.offset += toRead; + return toRead; + } + + @Override boolean tryReadAscii(char[] destination, int length) { + for (int i = 0; i < length; i++) { + byte b = buf[offset + i]; + if ((b & 0x80) != 0) return false; // Not 7-bit ASCII character + destination[i] = (char) b; + } + offset += length; + return true; + } + + @Override final String doReadUtf8(int length) { + String result = new String(buf, offset, length, UTF_8); + offset += length; + return result; + } + + @Override short readShort() { + require(2); + return (short) ((buf[offset++] & 0xff) << 8 | (buf[offset++] & 0xff)); + } + + @Override int readInt() { + require(4); + int pos = this.offset; + this.offset = pos + 4; + return (buf[pos] & 0xff) << 24 + | (buf[pos + 1] & 0xff) << 16 + | (buf[pos + 2] & 0xff) << 8 + | (buf[pos + 3] & 0xff); + } + + /** Code is optimized for little endian as proto is the encouraged format. */ + @Override long readLong() { + return Long.reverseBytes(readLongLe()); + } + + @Override long readLongLe() { + require(8); + int pos = this.offset; + this.offset = 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; + } + + @Override public int pos() { + return offset; + } + + @Override public long skip(long maxCount) { + int toSkip = Math.min(available(), (int) maxCount); + offset += toSkip; + return toSkip; + } + + @Override public int available() { + return length - offset; + } + } + + @Override public abstract int read(byte[] dst, int offset, int length); + + @Override public abstract long skip(long n); + + @Override public abstract int available(); + + @Override public void close() { + } + + @Override public void mark(int readlimit) { + throw new UnsupportedOperationException(); + } + + @Override public synchronized void reset() { + throw new UnsupportedOperationException(); + } + + @Override public boolean markSupported() { + return false; + } + + /** only use when you've already ensured the length you need is available */ + abstract byte readByteUnsafe(); + + final byte readByte() { + require(1); + return readByteUnsafe(); + } + + abstract byte[] readBytes(int length); + + final String readUtf8(int length) { + if (length == 0) return ""; // ex error tag with no value + require(length); + if (length > Platform.SHORT_STRING_LENGTH) return doReadUtf8(length); + + // Speculatively assume all 7-bit ASCII characters.. common in normal tags and names + char[] buffer = Platform.shortStringBuffer(); + if (tryReadAscii(buffer, length)) { + return new String(buffer, 0, length); + } + return doReadUtf8(length); + } + + abstract boolean tryReadAscii(char[] destination, int length); + + abstract String doReadUtf8(int length); + + abstract int pos(); + + abstract short readShort(); + + abstract int readInt(); + + abstract long readLong(); + + abstract long readLongLe(); + + @Override public final int read() { + return available() > 0 ? readByteUnsafe() : -1; + } + + final 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 = readByteUnsafe(); + result[i + 0] = HEX_DIGITS[(b >> 4) & 0xf]; + result[i + 1] = HEX_DIGITS[b & 0xf]; + } + return new String(result, 0, hexLength); + } + + /** + * @return the value read. Use {@link WriteBuffer#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 + final 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; + } + + final 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; + } + + final void require(int byteCount) { + if (this.available() < byteCount) { + throw new IllegalArgumentException( + "Truncated: length " + byteCount + " > bytes available " + this.available()); + } + } + + int checkReadArguments(byte[] dst, int offset, int length) { + if (dst == null) throw new NullPointerException(); + if (offset < 0 || length < 0 || length > dst.length - offset) { + throw new IndexOutOfBoundsException(); + } + return Math.min(available(), length); + } +} diff --git a/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java b/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java index 258c5a515d8..1d1abe9cd65 100644 --- a/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java +++ b/zipkin/src/main/java/zipkin2/internal/ThriftCodec.java @@ -18,8 +18,6 @@ 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_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 byte[] write(Span span) { } /** Encoding overhead is thrift type plus 32-bit length prefix */ - static int listSizeInBytes(UnsafeBuffer.Writer writer, List values) { + static int listSizeInBytes(WriteBuffer.Writer writer, List values) { int sizeInBytes = 5; for (int i = 0, length = values.size(); i < length; i++) { sizeInBytes += writer.sizeInBytes(values.get(i)); @@ -66,9 +62,8 @@ static int listSizeInBytes(UnsafeBuffer.Writer writer, List values) { return sizeInBytes; } - public static boolean read(byte[] bytes, Collection out) { - if (bytes.length == 0) return false; - ByteBuffer buffer = ByteBuffer.wrap(bytes); + public static boolean read(ReadBuffer buffer, Collection out) { + if (buffer.available() == 0) return false; try { V1Span v1Span = new V1ThriftSpanReader().read(buffer); V1SpanConverter.create().convert(v1Span, out); @@ -79,9 +74,8 @@ public static boolean read(byte[] bytes, Collection out) { } @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 out = new ArrayList<>(1); @@ -92,10 +86,9 @@ public static Span readOne(byte[] bytes) { } } - public static boolean readList(byte[] bytes, Collection out) { - int length = bytes.length; + public static boolean readList(ReadBuffer buffer, Collection 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 static boolean readList(byte[] bytes, Collection out) { 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 void writeList(UnsafeBuffer.Writer writer, List value, UnsafeBuffer buffer) { + static void writeList(WriteBuffer.Writer writer, List value, WriteBuffer buffer) { int length = value.size(); writeListBegin(buffer, length); for (int i = 0; i < length; i++) { @@ -134,50 +127,49 @@ static IllegalArgumentException exceptionReading(String type, Exception e) { 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 @@ static void skip(ByteBuffer bytes, byte type, int maxDepth) { } } - 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 fd646ca6fdd..c2527d21a57 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 @@ static Endpoint read(ByteBuffer bytes) { }); } } 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 @@ static int sizeInBytes(Endpoint value) { 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 e0d62b4e3a8..0088de101f6 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/V1JsonSpanReader.java b/zipkin/src/main/java/zipkin2/internal/V1JsonSpanReader.java index 921a0b71047..fff6698732b 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.Builder builder; - public boolean readList(byte[] bytes, Collection out) { - if (bytes.length == 0) return false; + public boolean readList(ReadBuffer buffer, Collection 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 boolean readList(byte[] bytes, Collection out) { } } - @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 @@ void readAnnotation(JsonReader reader) throws IOException { 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 d16c7e13326..d9c904b9b1e 100644 --- a/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java +++ b/zipkin/src/main/java/zipkin2/internal/V1JsonSpanWriter.java @@ -22,18 +22,16 @@ /** 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 { +public final class V1JsonSpanWriter implements WriteBuffer.Writer { 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 d891ca3477b..9bb19c5d39d 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.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 { +public final class V1SpanWriter implements WriteBuffer.Writer { - @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 int sizeInBytes(V1Span value) { 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 void write(V1Span value, UnsafeBuffer b) { 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), true); + return result; } static int binaryAnnotationSizeInBytes(String key, String value, int endpointSize) { @@ -209,7 +206,8 @@ static int binaryAnnotationSizeInBytes(String key, String value, int endpointSiz 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 541e0ccfb81..acb4de4a04d 100644 --- a/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanReader.java +++ b/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanReader.java @@ -16,14 +16,10 @@ */ 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.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; @@ -42,13 +38,15 @@ import static zipkin2.internal.V1ThriftSpanWriter.TRACE_ID_HIGH; public final class V1ThriftSpanReader { + static final String ONE = Character.toString((char) 1); + public static V1ThriftSpanReader create() { return new 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 +56,37 @@ public V1Span read(ByteBuffer bytes) { 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 +98,24 @@ static final class AnnotationReader { 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 +130,7 @@ static final class BinaryAnnotationReader { 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 +138,14 @@ static void read(ByteBuffer bytes, V1Span.Builder builder) { 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 +154,9 @@ static void read(ByteBuffer bytes, V1Span.Builder builder) { 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 +170,6 @@ static void read(ByteBuffer bytes, V1Span.Builder builder) { } } - V1ThriftSpanReader() {} + V1ThriftSpanReader() { + } } diff --git a/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java b/zipkin/src/main/java/zipkin2/internal/V1ThriftSpanWriter.java index 5c80d1c4d4b..ab57573d525 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.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_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 { - +public final class V1ThriftSpanWriter implements WriteBuffer.Writer { 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 { 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 int sizeInBytes(Span value) { 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 void write(Span value, UnsafeBuffer buffer) { 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 @@ static void writeAnnotations(UnsafeBuffer buffer, V1Span v1Span, byte[] endpoint } } - 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 @@ static void writeBinaryAnnotations(UnsafeBuffer buffer, V1Span v1Span, byte[] en } } - @Override - public String toString() { + @Override public String toString() { return "Span"; } @@ -170,22 +166,22 @@ public byte[] writeList(List spans) { 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)); + 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)); + return result; } public int writeList(List 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 int writeList(List spans, byte[] out, int pos) { 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)); + return result; } static class ThriftAnnotationWriter { @@ -213,7 +209,7 @@ static int sizeInBytes(int valueSizeInBytes, int endpointSizeInBytes) { 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 @@ static int sizeInBytes(int keySize, int valueSize, int endpointSizeInBytes) { 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 9dbba53e8d0..05ca43aecb5 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.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 { - @Override - public int sizeInBytes(Span value) { +public final class V2SpanWriter implements WriteBuffer.Writer { + @Override public int sizeInBytes(Span value) { int sizeInBytes = 13; // {"traceId":"" sizeInBytes += value.traceId().length(); if (value.parentId() != null) { @@ -88,8 +87,7 @@ public int sizeInBytes(Span value) { 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 void write(Span value, UnsafeBuffer b) { b.writeByte('}'); } - @Override - public String toString() { + @Override public String toString() { return "Span"; } @@ -192,7 +189,7 @@ static int endpointSizeInBytes(Endpoint value, boolean writeEmptyServiceName) { 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 @@ static int annotationSizeInBytes(long timestamp, String value, int endpointSizeI 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/main/java/zipkin2/internal/UnsafeBuffer.java b/zipkin/src/main/java/zipkin2/internal/WriteBuffer.java similarity index 65% rename from zipkin/src/main/java/zipkin2/internal/UnsafeBuffer.java rename to zipkin/src/main/java/zipkin2/internal/WriteBuffer.java index 0ae5d7fb1d5..d810601d834 100644 --- a/zipkin/src/main/java/zipkin2/internal/UnsafeBuffer.java +++ b/zipkin/src/main/java/zipkin2/internal/WriteBuffer.java @@ -16,36 +16,32 @@ */ package zipkin2.internal; -import static zipkin2.internal.JsonCodec.UTF_8; +import static zipkin2.internal.HexCodec.HEX_DIGITS; /** - *

    Read operations do bounds checks, as typically more errors occur reading than writing. - * - *

    Writes are unsafe as they do no bounds checks. This means you should take care to allocate or + * 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 final class WriteBuffer { + public interface Writer { + int sizeInBytes(T value); - public static UnsafeBuffer wrap(byte[] bytes, int pos) { - return new UnsafeBuffer(bytes, pos); + void write(T value, WriteBuffer buffer); } - public static UnsafeBuffer allocate(int sizeInBytes) { - return new UnsafeBuffer(sizeInBytes); + public static WriteBuffer wrap(byte[] bytes) { + return wrap(bytes, 0); } - private final byte[] buf; - int pos; // visible for testing - - UnsafeBuffer(int size) { - buf = new byte[size]; + public static WriteBuffer wrap(byte[] bytes, int pos) { + return new WriteBuffer(bytes, pos); } - UnsafeBuffer(byte[] buf, int pos) { + final byte[] buf; + int pos; + + WriteBuffer(byte[] buf, int pos) { this.buf = buf; this.pos = pos; } @@ -70,7 +66,7 @@ void writeBackwards(long v) { } /** Inspired by {@code okio.Buffer.writeLong} */ - public void writeLongHex(long v) { + void writeLongHex(long v) { int pos = this.pos; writeHexByte(buf, pos + 0, (byte) ((v >>> 56L) & 0xff)); writeHexByte(buf, pos + 2, (byte) ((v >>> 48L) & 0xff)); @@ -83,185 +79,15 @@ public void writeLongHex(long v) { 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. - * - *

    Originally based on - * http://stackoverflow.com/questions/8511490/calculating-length-in-utf-8-of-java-string-without-actually-encoding-it - * - *

    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. - * - *

    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. - * - *

    See https://developers.google.com/protocol-buffers/docs/encoding#varints - * - *

    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]; } + final int pos() { + return pos; + } + public void writeAscii(String v) { for (int i = 0, length = v.length(); i < length; i++) { writeByte(v.charAt(i) & 0xff); @@ -322,7 +148,6 @@ public void writeUtf8(CharSequence string) { // Adapted from okio.Buffer.writeDecimalLong public void writeAscii(long v) { if (v == 0) { - require(1); writeByte('0'); return; } @@ -369,86 +194,106 @@ void writeLongLe(long v) { 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 + * This returns the bytes needed to transcode a UTF-16 Java String to UTF-8 bytes. + * + *

    Originally based on + * http://stackoverflow.com/questions/8511490/calculating-length-in-utf-8-of-java-string-without-actually-encoding-it + * + *

    Later, ASCII run and malformed surrogate logic borrowed from okio.Utf8 */ - // 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)); + // TODO: benchmark vs https://github.com/protocolbuffers/protobuf/blob/master/java/core/src/main/java/com/google/protobuf/Utf8.java#L240 + // there seem to be less branches for for strings without surrogates + 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 result | b << 28; + return sizeInBytes; } - long readVarint64() { - byte b; // negative number implies MSB set - if ((b = readByte()) >= 0) { - return b; - } + /** + * Binary search for character width which favors matching lower numbers. + * + *

    Adapted from okio.Buffer + */ + public static int asciiSizeInBytes(long v) { + if (v == 0) return 1; + if (v == Long.MIN_VALUE) return 20; - 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); + boolean negative = false; + if (v < 0) { + v = -v; // making this positive allows us to compare using less-than + negative = true; } - return result; + 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 } - public interface Writer { - int sizeInBytes(T value); - - void write(T value, UnsafeBuffer buffer); + /** + * A base 128 varint encodes 7 bits at a time, this checks how many bytes are needed to represent + * the value. + * + *

    See https://developers.google.com/protocol-buffers/docs/encoding#varints + * + *

    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; } - void require(int byteCount) { - if (pos + byteCount > buf.length) { - throw new IllegalArgumentException( - "Truncated: length " + byteCount + " > bytes remaining " + remaining()); - } + /** Like {@link #varintSizeInBytes(int)}, except for uint64. */ + // TODO: benchmark vs https://github.com/protocolbuffers/protobuf/blob/master/java/core/src/main/java/com/google/protobuf/CodedOutputStream.java#L770 + // Since trace IDs are random, I guess they cover the entire spectrum of varint sizes and probably would especially benefit from this. + 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; } } diff --git a/zipkin/src/test/java/zipkin2/codec/SpanBytesDecoderTest.java b/zipkin/src/test/java/zipkin2/codec/SpanBytesDecoderTest.java index 679c3e6b641..6f2ef577955 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 from proto3"); + thrown.expectMessage("Truncated: length 66 > bytes available 8 reading List 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 from proto3"); + thrown.expectMessage("Truncated: length 101 > bytes available 3 reading List 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 4d66efa0a97..d4af20d699d 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 from TBinary"); + thrown.expectMessage("Truncated: length 8 > bytes available 2 reading List 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 void niceErrorOnMalformed_inputSpans_JSON_V1() { @Test public void niceErrorOnMalformed_inputSpans_THRIFT() { thrown.expect(IllegalArgumentException.class); - thrown.expectMessage( - "Truncated: length 1701604463 > bytes remaining 0 reading List from TBinary"); + thrown.expectMessage("Truncated: length 1 > bytes available 0 reading List 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 bc6c60e9b04..7dd9c787696 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 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 void dependenciesRoundTrip() { 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 5421518f041..e8fc3e987b3 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 @@ class Foo { 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 @@ class Foo { Exception error = null; byte[] bytes = "[\"='".getBytes(UTF_8); try { - new JsonCodec.JsonReader(bytes).beginObject(); + new JsonCodec.JsonReader(ReadBuffer.wrap(bytes)).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 2893dc5ca10..028eda5a27a 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_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); /** 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 */, bytes.length); + 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 */, bytes.length); + 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 */, bytes.length); + 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 */, bytes.length); + 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 */, bytes.length); + 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).writeVarint(100); // much larger than the buffer size try { - field.readLengthPrefixAndValue(buf); + field.readLengthPrefixAndValue(ReadBuffer.wrap(bytes)); 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 */, bytes.length); + 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 03c31de981f..2925951a38d 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.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 ad3d139ba9a..834c0d79c92 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.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); /** 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))) .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))) .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 @@ static Span.Builder spanBuilder() { void assertRoundTrip(Span span) { SPAN.write(buf, span); - buf.reset(); - assertThat(SPAN.read(buf)) + assertThat(SPAN.read(ReadBuffer.wrap(bytes))) .isEqualTo(span); } } diff --git a/zipkin/src/test/java/zipkin2/internal/ReadBufferTest.java b/zipkin/src/test/java/zipkin2/internal/ReadBufferTest.java new file mode 100644 index 00000000000..3b0b77d80ba --- /dev/null +++ b/zipkin/src/test/java/zipkin2/internal/ReadBufferTest.java @@ -0,0 +1,239 @@ +/* + * 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.ByteOrder; +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 ReadBufferTest { + @Test public void byteBuffer_limited() { + ByteBuffer buf = ByteBuffer.wrap("glove".getBytes(UTF_8)); + buf.get(); + ReadBuffer readBuffer = ReadBuffer.wrapUnsafe(buf.slice()); + assertThat(readBuffer.readUtf8(readBuffer.available())) + .isEqualTo("love"); + } + + @Test public void readVarint32() { + assertReadVarint32(0); + assertReadVarint32(0b0011_1111_1111_1111); + assertReadVarint32(0xFFFFFFFF); + } + + static void assertReadVarint32(int value) { + byte[] bytes = new byte[WriteBuffer.varintSizeInBytes(value)]; + WriteBuffer.wrap(bytes).writeVarint(value); + + assertThat(ReadBuffer.wrap(bytes).readVarint32()) + .isEqualTo(value); + } + + @Test public void readShort_bytes() { + byte[] bytes = {(byte) 0x01, (byte) 0x02}; + + ReadBuffer readBuffer = ReadBuffer.wrap(bytes); + + assertThat(readBuffer.readShort()).isEqualTo((short) 0x0102); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readShort_byteBuff() { + byte[] bytes = {(byte) 0x01, (byte) 0x02}; + + ByteBuffer buffer = ByteBuffer.wrap(bytes).asReadOnlyBuffer(); + ReadBuffer readBuffer = ReadBuffer.wrapUnsafe(buffer); + assertThat(readBuffer).isInstanceOf(ReadBuffer.Buff.class); + + assertThat(readBuffer.readShort()).isEqualTo((short) 0x0102); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readShort_byteBuff_littleEndian() { + byte[] bytes = {(byte) 0x01, (byte) 0x02}; + + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).asReadOnlyBuffer(); + ReadBuffer readBuffer = ReadBuffer.wrapUnsafe(buffer); + assertThat(readBuffer).isInstanceOf(ReadBuffer.Buff.class); + + assertThat(readBuffer.readShort()).isEqualTo((short) 0x0102); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readInt_bytes() { + byte[] bytes = {(byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04}; + + ReadBuffer readBuffer = ReadBuffer.wrap(bytes); + + assertThat(readBuffer.readInt()).isEqualTo(0x01020304); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readInt_byteBuff() { + byte[] bytes = {(byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04}; + + ByteBuffer buffer = ByteBuffer.wrap(bytes).asReadOnlyBuffer(); + ReadBuffer readBuffer = ReadBuffer.wrapUnsafe(buffer); + assertThat(readBuffer).isInstanceOf(ReadBuffer.Buff.class); + + assertThat(readBuffer.readInt()).isEqualTo(0x01020304); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readInt_byteBuff_littleEndian() { + byte[] bytes = {(byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04}; + + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).asReadOnlyBuffer(); + ReadBuffer readBuffer = ReadBuffer.wrapUnsafe(buffer); + assertThat(readBuffer).isInstanceOf(ReadBuffer.Buff.class); + + assertThat(readBuffer.readInt()).isEqualTo(0x01020304); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readLong_bytes() { + byte[] bytes = { + (byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04, + (byte) 0x05, (byte) 0x06, (byte) 0x07, (byte) 0x08, + }; + + ReadBuffer readBuffer = ReadBuffer.wrap(bytes); + + assertThat(readBuffer.readLong()) + .isEqualTo(0x0102030405060708L); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readLong_byteBuff() { + byte[] bytes = { + (byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04, + (byte) 0x05, (byte) 0x06, (byte) 0x07, (byte) 0x08, + }; + + ByteBuffer buffer = ByteBuffer.wrap(bytes).asReadOnlyBuffer(); + ReadBuffer readBuffer = ReadBuffer.wrapUnsafe(buffer); + assertThat(readBuffer).isInstanceOf(ReadBuffer.Buff.class); + + assertThat(readBuffer.readLong()) + .isEqualTo(0x0102030405060708L); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readLong_byteBuff_littleEndian() { + byte[] bytes = { + (byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04, + (byte) 0x05, (byte) 0x06, (byte) 0x07, (byte) 0x08, + }; + + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).asReadOnlyBuffer(); + ReadBuffer readBuffer = ReadBuffer.wrapUnsafe(buffer); + assertThat(readBuffer).isInstanceOf(ReadBuffer.Buff.class); + + assertThat(readBuffer.readLong()) + .isEqualTo(0x0102030405060708L); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readLongLe_bytes() { + byte[] bytes = { + (byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04, + (byte) 0x05, (byte) 0x06, (byte) 0x07, (byte) 0x08, + }; + + ReadBuffer readBuffer = ReadBuffer.wrap(bytes); + + assertThat(readBuffer.readLongLe()) + .isEqualTo(0x0807060504030201L); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readLongLe_byteBuff() { + byte[] bytes = { + (byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04, + (byte) 0x05, (byte) 0x06, (byte) 0x07, (byte) 0x08, + }; + + ByteBuffer buffer = ByteBuffer.wrap(bytes).asReadOnlyBuffer(); + ReadBuffer readBuffer = ReadBuffer.wrapUnsafe(buffer); + assertThat(readBuffer).isInstanceOf(ReadBuffer.Buff.class); + + assertThat(readBuffer.readLongLe()) + .isEqualTo(0x0807060504030201L); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readLongLe_byteBuff_littleEndian() { + byte[] bytes = { + (byte) 0x01, (byte) 0x02, (byte) 0x03, (byte) 0x04, + (byte) 0x05, (byte) 0x06, (byte) 0x07, (byte) 0x08, + }; + + ByteBuffer buffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).asReadOnlyBuffer(); + ReadBuffer readBuffer = ReadBuffer.wrapUnsafe(buffer); + assertThat(readBuffer).isInstanceOf(ReadBuffer.Buff.class); + + assertThat(readBuffer.readLongLe()) + .isEqualTo(0x0807060504030201L); + assertThat(readBuffer.available()).isZero(); + } + + @Test public void readVarint32_malformedTooBig() { + byte[] bytes = new byte[8]; + WriteBuffer.wrap(bytes).writeLongLe(0xffffffffffffL); + + try { + ReadBuffer.wrap(bytes).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) { + byte[] bytes = new byte[WriteBuffer.varintSizeInBytes(value)]; + WriteBuffer.wrap(bytes).writeVarint(value); + + assertThat(ReadBuffer.wrap(bytes).readVarint64()) + .isEqualTo(value); + } + + @Test public void readVarint64_malformedTooBig() { + byte[] bytes = new byte[16]; + WriteBuffer buffer = WriteBuffer.wrap(bytes); + buffer.writeLongLe(0xffffffffffffffffL); + buffer.writeLongLe(0xffffffffffffffffL); + + try { + ReadBuffer.wrap(bytes).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 7f6ae0abc21..f95dfae2729 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); @Test public void sizeInBytes() { @@ -37,153 +37,149 @@ public void sizeInBytes() { } @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 void writesEmptySpanName() throws IOException { 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 void writesEmptyServiceName() throws IOException { 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 ee32150f45a..fba72174b05 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 @@ 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); 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)).portAsInt()) .isEqualTo(highPort); } @@ -72,8 +67,8 @@ public void write_startsWithI64Prefix() { 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 void writeList_startsWithListPrefix() { 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 void writeList_startsWithListPrefix_multiple() { 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 void writeList_empty() { @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)); - assertThat(buf.unwrap()).containsExactly(buf.unwrap()); + assertThat(bytes).containsExactly(bytes2); } @Test @@ -144,12 +139,12 @@ void writesCoreAnnotationsNoEndpoint(Span.Kind kind, String begin, String end) { 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 @@ void writesBeginAnnotationNoEndpoint(Span.Kind kind, String begin) { 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 void writesAddressBinaryAnnotation_consumer() { 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 void writesEmptySpanName() { 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 void writesEmptyAnnotationAndBinaryAnnotations() { 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 void writesEmptyLocalComponentWhenNoAnnotationsOrTags() { 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 void writesDebugFalse() { 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 8c632ed863d..415cb76b198 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 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); @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\"}"); } } diff --git a/zipkin/src/test/java/zipkin2/internal/UnsafeBufferTest.java b/zipkin/src/test/java/zipkin2/internal/WriteBufferTest.java similarity index 52% rename from zipkin/src/test/java/zipkin2/internal/UnsafeBufferTest.java rename to zipkin/src/test/java/zipkin2/internal/WriteBufferTest.java index 22f97f2180c..941191e899b 100644 --- a/zipkin/src/test/java/zipkin2/internal/UnsafeBufferTest.java +++ b/zipkin/src/test/java/zipkin2/internal/WriteBufferTest.java @@ -16,17 +16,15 @@ */ 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 { +public class WriteBufferTest { // Adapted from http://stackoverflow.com/questions/8511490/calculating-length-in-utf-8-of-java-string-without-actually-encoding-it @Test public void utf8SizeInBytes() { for (int codepoint = 0; codepoint <= 0x10FFFF; codepoint++) { @@ -34,7 +32,7 @@ public class UnsafeBufferTest { if (Character.isDefined(codepoint)) { String test = new String(Character.toChars(codepoint)); int expected = test.getBytes(UTF_8).length; - int actual = UnsafeBuffer.utf8SizeInBytes(test); + int actual = WriteBuffer.utf8SizeInBytes(test); if (actual != expected) { throw new AssertionError(actual + " length != " + expected + " for " + codepoint); } @@ -46,12 +44,12 @@ public class UnsafeBufferTest { @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)) + assertThat(WriteBuffer.utf8SizeInBytes(test)) .isEqualTo(3); - UnsafeBuffer buffer = UnsafeBuffer.allocate(3); - buffer.writeUtf8(test); - assertThat(buffer.unwrap()) + byte[] bytes = new byte[3]; + WriteBuffer.wrap(bytes).writeUtf8(test); + assertThat(bytes) .containsExactly('a', '?', 'c'); } } @@ -61,12 +59,12 @@ public class UnsafeBufferTest { 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)) + assertThat(WriteBuffer.utf8SizeInBytes(test)) .isEqualTo(5); - UnsafeBuffer buffer = UnsafeBuffer.allocate(5); - buffer.writeUtf8(test); - assertThat(new String(buffer.unwrap(), UTF_8)) + byte[] bytes = new byte[5]; + WriteBuffer.wrap(bytes).writeUtf8(test); + assertThat(new String(bytes, UTF_8)) .isEqualTo("\uD83C\uDC00?"); } @@ -75,12 +73,12 @@ public class UnsafeBufferTest { char[] array = "\uD83C\uDC00\uD83C\uDC01".toCharArray(); array[array.length - 1] = 'c'; String test = new String(array); - assertThat(UnsafeBuffer.utf8SizeInBytes(test)) + assertThat(WriteBuffer.utf8SizeInBytes(test)) .isEqualTo(6); - UnsafeBuffer buffer = UnsafeBuffer.allocate(6); - buffer.writeUtf8(test); - assertThat(new String(buffer.unwrap(), UTF_8)) + byte[] bytes = new byte[6]; + WriteBuffer.wrap(bytes).writeUtf8(test); + assertThat(new String(bytes, UTF_8)) .isEqualTo("\uD83C\uDC00?c"); } @@ -91,43 +89,42 @@ public class UnsafeBufferTest { "ю́ cyrillic small letter yu with acute", "∃y ∀x ¬(x ≺ y)" )) { - int encodedSize = UnsafeBuffer.utf8SizeInBytes(string); + int encodedSize = WriteBuffer.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)) + byte[] bytes = new byte[encodedSize]; + WriteBuffer.wrap(bytes).writeUtf8(string); + assertThat(new String(bytes, UTF_8)) .isEqualTo(string); } } - @Test public void utf8_matchesAscii() throws Exception { + @Test public void utf8_matchesAscii() { String ascii = "86154a4ba6e913854d1e00c0db9010db"; - int encodedSize = UnsafeBuffer.utf8SizeInBytes(ascii); + int encodedSize = WriteBuffer.utf8SizeInBytes(ascii); assertThat(encodedSize) .isEqualTo(ascii.length()); - UnsafeBuffer bufferAscii = UnsafeBuffer.allocate(encodedSize); - bufferAscii.writeAscii(ascii); - assertThat(new String(bufferAscii.unwrap(), "US-ASCII")) + byte[] bytes = new byte[encodedSize]; + WriteBuffer.wrap(bytes).writeAscii(ascii); + assertThat(new String(bytes, UTF_8)) .isEqualTo(ascii); - UnsafeBuffer bufferUtf8 = UnsafeBuffer.allocate(encodedSize); - bufferUtf8.writeUtf8(ascii); - assertThat(new String(bufferUtf8.unwrap(), "US-ASCII")) + WriteBuffer.wrap(bytes).writeUtf8(ascii); + assertThat(new String(bytes, UTF_8)) .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)) + assertThat(WriteBuffer.utf8SizeInBytes(emoji)) .isEqualTo(emojiBytes.length); - UnsafeBuffer buffer = UnsafeBuffer.allocate(emojiBytes.length); - buffer.writeUtf8(emoji); - assertThat(buffer.unwrap()) + byte[] bytes = new byte[emojiBytes.length]; + WriteBuffer.wrap(bytes).writeUtf8(emoji); + assertThat(bytes) .isEqualTo(emojiBytes); } @@ -143,45 +140,45 @@ public class UnsafeBufferTest { } static String writeAscii(long v) { - UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.asciiSizeInBytes(v)); - buffer.writeAscii(v); - return new String(buffer.unwrap(), UTF_8); + byte[] bytes = new byte[WriteBuffer.asciiSizeInBytes(v)]; + WriteBuffer.wrap(bytes).writeAscii(v); + return new String(bytes, UTF_8); } // Test creating Buffer for a long string - @Test public void writeString() throws UnsupportedEncodingException { - StringBuffer stringBuffer = new StringBuffer(); + @Test public void writeString() { + StringBuilder builder = new StringBuilder(); for (int i = 0; i < 100000; i++) { - stringBuffer.append("a"); + builder.append("a"); } - String string = stringBuffer.toString(); - UnsafeBuffer buffer = UnsafeBuffer.allocate(string.length()); - buffer.writeAscii(string); - assertThat(new String(buffer.unwrap(), "US-ASCII")).isEqualTo(string); + String string = builder.toString(); + byte[] bytes = new byte[string.length()]; + WriteBuffer.wrap(bytes).writeAscii(string); + assertThat(new String(bytes, UTF_8)).isEqualTo(string); } @Test public void unsignedVarintSize_32_largest() { // largest to encode is a negative number - assertThat(UnsafeBuffer.varintSizeInBytes(Integer.MIN_VALUE)) + assertThat(WriteBuffer.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)) + assertThat(WriteBuffer.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); + byte[] bytes = new byte[8]; + WriteBuffer.wrap(bytes).writeLongLe(number); ByteBuffer byteBuffer = ByteBuffer.allocate(8); byteBuffer.order(ByteOrder.LITTLE_ENDIAN); byteBuffer.putLong(number); - assertThat(buffer.unwrap()) + assertThat(bytes) .containsExactly(byteBuffer.array()); } } @@ -190,10 +187,10 @@ static String writeAscii(long v) { @Test public void writeVarint_32() { int number = 300; - UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(number)); - buffer.writeVarint(number); + byte[] bytes = new byte[WriteBuffer.varintSizeInBytes(number)]; + WriteBuffer.wrap(bytes).writeVarint(number); - assertThat(buffer.unwrap()) + assertThat(bytes) .containsExactly(0b1010_1100, 0b0000_0010); } @@ -201,100 +198,41 @@ static String writeAscii(long v) { @Test public void writeVarint_64() { long number = 300; - UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(number)); - buffer.writeVarint(number); + byte[] bytes = new byte[WriteBuffer.varintSizeInBytes(number)]; + WriteBuffer.wrap(bytes).writeVarint(number); - assertThat(buffer.unwrap()) + assertThat(bytes) .containsExactly(0b1010_1100, 0b0000_0010); } @Test public void writeVarint_ports() { // normal case - UnsafeBuffer buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(80)); - buffer.writeVarint(80); + byte[] bytes = new byte[WriteBuffer.varintSizeInBytes(80)]; + WriteBuffer.wrap(bytes).writeVarint(80); - assertThat(buffer.unwrap()) + assertThat(bytes) .containsExactly(0b0101_0000); // largest value to not require more than 2 bytes (14 bits set) - buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(16383)); - buffer.writeVarint(16383); + bytes = new byte[WriteBuffer.varintSizeInBytes(16383)]; + WriteBuffer.wrap(bytes).writeVarint(16383); - assertThat(buffer.unwrap()) + assertThat(bytes) .containsExactly(0b1111_1111, 0b0111_1111); // worst case is a byte longer than fixed 16 - buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(65535)); - buffer.writeVarint(65535); + bytes = new byte[WriteBuffer.varintSizeInBytes(65535)]; + WriteBuffer.wrap(bytes).writeVarint(65535); - assertThat(buffer.unwrap()) + assertThat(bytes) .containsExactly(0b1111_1111, 0b1111_1111, 0b0000_0011); // most bits - buffer = UnsafeBuffer.allocate(UnsafeBuffer.varintSizeInBytes(0xFFFFFFFF)); - buffer.writeVarint(0xFFFFFFFF); + bytes = new byte[WriteBuffer.varintSizeInBytes(0xFFFFFFFF)]; + WriteBuffer.wrap(bytes).writeVarint(0xFFFFFFFF); // we have a total of 32 bits encoded - assertThat(buffer.unwrap()) + assertThat(bytes) .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"); - } - } }