Skip to content

Commit 318ce93

Browse files
committed
Use Java APIs instead of ByteStreams
1 parent c767dad commit 318ce93

File tree

19 files changed

+68
-78
lines changed

19 files changed

+68
-78
lines changed

core/trino-main/src/main/java/io/trino/execution/buffer/PagesSerdeUtil.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,6 @@
1414
package io.trino.execution.buffer;
1515

1616
import com.google.common.collect.AbstractIterator;
17-
import com.google.common.io.ByteStreams;
1817
import io.airlift.slice.Slice;
1918
import io.airlift.slice.SliceInput;
2019
import io.airlift.slice.SliceOutput;
@@ -33,7 +32,7 @@
3332
import java.util.List;
3433

3534
import static com.google.common.base.Preconditions.checkArgument;
36-
import static com.google.common.io.ByteStreams.readFully;
35+
import static com.google.common.base.Verify.verify;
3736
import static io.trino.block.BlockSerdeUtil.readBlock;
3837
import static io.trino.block.BlockSerdeUtil.writeBlock;
3938
import static io.trino.execution.buffer.PageCodecMarker.COMPRESSED;
@@ -158,7 +157,7 @@ private static class PageReader
158157
protected Page computeNext()
159158
{
160159
try {
161-
int read = ByteStreams.read(inputStream, headerBuffer, 0, headerBuffer.length);
160+
int read = inputStream.readNBytes(headerBuffer, 0, headerBuffer.length);
162161
if (read <= 0) {
163162
return endOfData();
164163
}
@@ -195,7 +194,7 @@ private static class SerializedPageReader
195194
protected Slice computeNext()
196195
{
197196
try {
198-
int read = ByteStreams.read(inputStream, headerBuffer, 0, headerBuffer.length);
197+
int read = inputStream.readNBytes(headerBuffer, 0, headerBuffer.length);
199198
if (read <= 0) {
200199
return endOfData();
201200
}
@@ -219,7 +218,8 @@ public static Slice readSerializedPage(Slice headerSlice, InputStream inputStrea
219218
int compressedSize = headerSlice.getIntUnchecked(SERIALIZED_PAGE_COMPRESSED_SIZE_OFFSET);
220219
byte[] outputBuffer = new byte[SERIALIZED_PAGE_HEADER_SIZE + compressedSize];
221220
headerSlice.getBytes(0, outputBuffer, 0, SERIALIZED_PAGE_HEADER_SIZE);
222-
readFully(inputStream, outputBuffer, SERIALIZED_PAGE_HEADER_SIZE, compressedSize);
221+
int bytes = inputStream.readNBytes(outputBuffer, SERIALIZED_PAGE_HEADER_SIZE, compressedSize);
222+
verify(bytes == compressedSize, "expected to read %s bytes, but read %s", compressedSize, bytes);
223223
return Slices.wrappedBuffer(outputBuffer);
224224
}
225225
}

core/trino-main/src/main/java/io/trino/server/ui/WorkerResource.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -49,7 +49,6 @@
4949
import java.util.Optional;
5050
import java.util.Set;
5151

52-
import static com.google.common.io.ByteStreams.toByteArray;
5352
import static com.google.common.net.HttpHeaders.CONTENT_TYPE;
5453
import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom;
5554
import static io.airlift.http.client.Request.Builder.prepareGet;
@@ -226,7 +225,7 @@ public byte[] handle(Request request, io.airlift.http.client.Response response)
226225
if (!APPLICATION_JSON.equals(response.getHeader(CONTENT_TYPE))) {
227226
throw new RuntimeException("Response received was not of type " + APPLICATION_JSON);
228227
}
229-
return toByteArray(response.getInputStream());
228+
return response.getInputStream().readAllBytes();
230229
}
231230
catch (IOException e) {
232231
throw new RuntimeException("Unable to read response from worker", e);

core/trino-main/src/main/java/io/trino/sql/gen/IsolatedClass.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,6 @@
1414
package io.trino.sql.gen;
1515

1616
import com.google.common.collect.ImmutableMap;
17-
import com.google.common.io.ByteStreams;
1817
import io.airlift.bytecode.DynamicClassLoader;
1918

2019
import java.io.IOException;
@@ -60,7 +59,7 @@ private static byte[] getBytecode(Class<?> clazz)
6059
{
6160
try (InputStream stream = clazz.getClassLoader().getResourceAsStream(clazz.getName().replace('.', '/') + ".class")) {
6261
checkArgument(stream != null, "Could not obtain byte code for class %s", clazz.getName());
63-
return ByteStreams.toByteArray(stream);
62+
return stream.readAllBytes();
6463
}
6564
catch (IOException e) {
6665
throw new RuntimeException(format("Could not obtain byte code for class %s", clazz.getName()), e);

core/trino-main/src/test/java/io/trino/operator/TestDirectExchangeClient.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,6 @@
7171
import static com.google.common.collect.ImmutableSet.toImmutableSet;
7272
import static com.google.common.collect.Maps.uniqueIndex;
7373
import static com.google.common.collect.Sets.newConcurrentHashSet;
74-
import static com.google.common.io.ByteStreams.toByteArray;
7574
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
7675
import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
7776
import static io.airlift.concurrent.MoreFutures.tryGetFutureValue;
@@ -887,7 +886,7 @@ public synchronized Response handle(Request request)
887886
checkState(response.getStatusCode() == HttpStatus.OK.code(), "Unexpected status code: %s", response.getStatusCode());
888887
ListMultimap<String, String> headers = response.getHeaders().entries().stream()
889888
.collect(toImmutableListMultimap(entry -> entry.getKey().toString(), Map.Entry::getValue));
890-
byte[] bytes = toByteArray(response.getInputStream());
889+
byte[] bytes = response.getInputStream().readAllBytes();
891890
checkState(bytes.length > 42, "too short");
892891
savedResponse = new TestingResponse(HttpStatus.OK, headers, bytes.clone());
893892
// corrupt

lib/trino-filesystem-s3/src/test/java/io/trino/filesystem/s3/AbstractTestS3FileSystem.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,6 @@
1414
package io.trino.filesystem.s3;
1515

1616
import com.google.common.collect.ImmutableList;
17-
import com.google.common.io.ByteStreams;
1817
import com.google.common.io.Closer;
1918
import io.airlift.log.Logging;
2019
import io.trino.filesystem.AbstractTestTrinoFileSystem;
@@ -172,7 +171,7 @@ void testFileWithTrailingWhitespaceAgainstNativeClient()
172171
TrinoInputFile inputFile = getFileSystem().newInputFile(fileEntry.location());
173172
assertThat(inputFile.exists()).as("exists").isTrue();
174173
try (TrinoInputStream inputStream = inputFile.newStream()) {
175-
byte[] bytes = ByteStreams.toByteArray(inputStream);
174+
byte[] bytes = inputStream.readAllBytes();
176175
assertThat(bytes).isEqualTo(contents);
177176
}
178177

lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,6 @@
1515

1616
import com.google.common.collect.ImmutableList;
1717
import com.google.common.collect.Ordering;
18-
import com.google.common.io.ByteStreams;
1918
import com.google.common.io.Closer;
2019
import io.airlift.slice.Slice;
2120
import io.airlift.units.Duration;
@@ -1153,7 +1152,7 @@ public void testFileWithTrailingWhitespace()
11531152
TrinoInputFile inputFile = getFileSystem().newInputFile(location);
11541153
assertThat(inputFile.exists()).as("exists").isTrue();
11551154
try (TrinoInputStream inputStream = inputFile.newStream()) {
1156-
byte[] bytes = ByteStreams.toByteArray(inputStream);
1155+
byte[] bytes = inputStream.readAllBytes();
11571156
assertThat(bytes).isEqualTo(("test blob content for " + location).getBytes(UTF_8));
11581157
}
11591158

@@ -1165,7 +1164,7 @@ public void testFileWithTrailingWhitespace()
11651164
// This can break some file system read operations (e.g., TrinoInput.readTail for most filesystems, newStream for caching file systems).
11661165
TrinoInputFile newInputFile = getFileSystem().newInputFile(location);
11671166
try (TrinoInputStream inputStream = newInputFile.newStream()) {
1168-
byte[] bytes = ByteStreams.toByteArray(inputStream);
1167+
byte[] bytes = inputStream.readAllBytes();
11691168
assertThat(bytes).isEqualTo(newContents);
11701169
}
11711170

@@ -1182,7 +1181,7 @@ public void testFileWithTrailingWhitespace()
11821181
assertThat(getFileSystem().newInputFile(target).exists()).as("target exists after rename").isTrue();
11831182

11841183
try (TrinoInputStream inputStream = getFileSystem().newInputFile(target).newStream()) {
1185-
byte[] bytes = ByteStreams.toByteArray(inputStream);
1184+
byte[] bytes = inputStream.readAllBytes();
11861185
assertThat(bytes).isEqualTo(("test blob content for " + source).getBytes(UTF_8));
11871186
}
11881187

lib/trino-hdfs/src/test/java/io/trino/hdfs/s3/TestTrinoS3FileSystem.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,6 @@
7474
import java.util.Map;
7575

7676
import static com.google.common.base.Preconditions.checkArgument;
77-
import static com.google.common.io.ByteStreams.toByteArray;
7877
import static com.google.common.io.MoreFiles.deleteRecursively;
7978
import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE;
8079
import static io.trino.hdfs.s3.TrinoS3FileSystem.NO_SUCH_BUCKET_ERROR_CODE;
@@ -894,7 +893,7 @@ public void testStreamingUpload()
894893
InputStream concatInputStream = parts.stream()
895894
.map(UploadPartRequest::getInputStream)
896895
.reduce(new ByteArrayInputStream(new byte[0]), SequenceInputStream::new);
897-
String data = new String(toByteArray(concatInputStream), US_ASCII);
896+
String data = new String(concatInputStream.readAllBytes(), US_ASCII);
898897
assertThat(data).isEqualTo("a" + "foo".repeat(21) + "bar".repeat(44) + "orange".repeat(22));
899898
}
900899
}

lib/trino-hive-formats/src/test/java/io/trino/hive/formats/TestTrinoDataInputStream.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,6 @@
1515

1616
import com.google.common.collect.ImmutableList;
1717
import com.google.common.io.ByteSource;
18-
import com.google.common.io.ByteStreams;
1918
import io.airlift.slice.Slice;
2019
import io.airlift.slice.Slices;
2120
import io.trino.filesystem.Location;
@@ -452,7 +451,7 @@ public String readActual(TrinoDataInputStream input)
452451
throws IOException
453452
{
454453
byte[] bytes = new byte[valueSize() + 10];
455-
ByteStreams.readFully(input, bytes, 5, valueSize());
454+
input.readFully(bytes, 5, valueSize());
456455
return new String(bytes, 5, valueSize(), UTF_8);
457456
}
458457
});
@@ -649,7 +648,7 @@ private static void testReadOffEnd(DataInputTester tester, byte[] bytes)
649648
throws IOException
650649
{
651650
TrinoDataInputStream input = createTrinoDataInputStream(bytes);
652-
ByteStreams.skipFully(input, bytes.length - tester.valueSize() + 1);
651+
input.skipNBytes(bytes.length - tester.valueSize() + 1);
653652
tester.verifyReadOffEnd(input);
654653
}
655654

lib/trino-parquet/src/main/java/io/trino/parquet/ParquetCompressionUtils.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,6 @@
1313
*/
1414
package io.trino.parquet;
1515

16-
import com.google.common.io.ByteStreams;
1716
import io.airlift.compress.v3.Decompressor;
1817
import io.airlift.compress.v3.lz4.Lz4Decompressor;
1918
import io.airlift.compress.v3.lzo.LzoDecompressor;
@@ -88,7 +87,7 @@ private static Slice decompressGzip(Slice input, int uncompressedSize)
8887

8988
try (GZIPInputStream gzipInputStream = new GZIPInputStream(input.getInput(), min(GZIP_BUFFER_SIZE, input.length()))) {
9089
byte[] buffer = new byte[uncompressedSize];
91-
int bytesRead = ByteStreams.read(gzipInputStream, buffer, 0, buffer.length);
90+
int bytesRead = gzipInputStream.readNBytes(buffer, 0, buffer.length);
9291
if (bytesRead != uncompressedSize) {
9392
throw new IllegalArgumentException(format("Invalid uncompressedSize for GZIP input. Expected %s, actual: %s", uncompressedSize, bytesRead));
9493
}

lib/trino-parquet/src/main/java/io/trino/parquet/reader/ChunkedInputStream.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@
2626

2727
import static com.google.common.base.Preconditions.checkArgument;
2828
import static com.google.common.base.Preconditions.checkPositionIndexes;
29-
import static com.google.common.io.ByteStreams.readFully;
29+
import static com.google.common.base.Verify.verify;
3030
import static io.airlift.slice.Slices.EMPTY_SLICE;
3131
import static java.util.Objects.requireNonNull;
3232

@@ -69,7 +69,8 @@ public Slice getSlice(int length)
6969
// requested length crosses the slice boundary
7070
byte[] bytes = new byte[length];
7171
try {
72-
readFully(this, bytes, 0, bytes.length);
72+
int read = this.readNBytes(bytes, 0, bytes.length);
73+
verify(read == length, "expected to read %s bytes but got %s", length, read);
7374
}
7475
catch (IOException e) {
7576
throw new RuntimeException("Failed to read " + length + " bytes", e);

0 commit comments

Comments
 (0)