From fb7005435b28ccca2ece3a6812734f2e8244cdea Mon Sep 17 00:00:00 2001 From: Prajwal Tuladhar Date: Mon, 23 Mar 2015 23:23:51 -0400 Subject: [PATCH] use ByteSink and ByteSource instead of OutputSupplier and InputSupplier They are being deprecated and will eventually be removed in Guava 18.0 --- .../storage/hdfs/HdfsDataSegmentPusher.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java b/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java index c3aa9f49edd..b1168ccfe70 100644 --- a/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java +++ b/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java @@ -19,8 +19,8 @@ package io.druid.storage.hdfs; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -import com.google.common.io.ByteStreams; -import com.google.common.io.OutputSupplier; +import com.google.common.io.ByteSink; +import com.google.common.io.ByteSource; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import io.druid.segment.SegmentUtils; @@ -75,7 +75,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher fs.mkdirs(outFile.getParent()); log.info("Compressing files from[%s] to [%s]", inDir, outFile); - long size; + final long size; try (FSDataOutputStream out = fs.create(outFile)) { size = CompressionUtils.zip(inDir, out); } @@ -93,10 +93,9 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher { final Path descriptorFile = new Path(outDir, "descriptor.json"); log.info("Creating descriptor file at[%s]", descriptorFile); - ByteStreams.copy( - ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), - new HdfsOutputStreamSupplier(fs, descriptorFile) - ); + ByteSource + .wrap(jsonMapper.writeValueAsBytes(segment)) + .copyTo(new HdfsOutputStreamSupplier(fs, descriptorFile)); return segment; } @@ -105,7 +104,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher return ImmutableMap.of("type", "hdfs", "path", outFile.toString()); } - private static class HdfsOutputStreamSupplier implements OutputSupplier + private static class HdfsOutputStreamSupplier extends ByteSink { private final FileSystem fs; private final Path descriptorFile; @@ -117,7 +116,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher } @Override - public OutputStream getOutput() throws IOException + public OutputStream openStream() throws IOException { return fs.create(descriptorFile); }