-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Support lz4 compression in hdfs #18982
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
75e5be6
1fa9666
cb2b03b
10299cb
66b5949
338eba1
c72fbba
e7b4c27
edf9a28
b556296
06b4775
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,6 +30,8 @@ | |
| import org.apache.druid.java.util.common.UOE; | ||
| import org.apache.druid.java.util.common.io.NativeIO; | ||
| import org.apache.druid.java.util.common.logger.Logger; | ||
| import org.apache.druid.java.util.emitter.service.ServiceEmitter; | ||
| import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; | ||
| import org.apache.druid.segment.loading.SegmentLoadingException; | ||
| import org.apache.druid.segment.loading.URIDataPuller; | ||
| import org.apache.druid.utils.CompressionUtils; | ||
|
|
@@ -40,6 +42,7 @@ | |
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hadoop.fs.RemoteIterator; | ||
|
|
||
| import javax.annotation.Nullable; | ||
| import javax.tools.FileObject; | ||
| import java.io.File; | ||
| import java.io.IOException; | ||
|
|
@@ -177,11 +180,21 @@ public boolean delete() | |
|
|
||
| private static final Logger log = new Logger(HdfsDataSegmentPuller.class); | ||
| protected final Configuration config; | ||
| private final ServiceEmitter emitter; | ||
|
|
||
| @Inject | ||
| public HdfsDataSegmentPuller(@Hdfs final Configuration config) | ||
| { | ||
| this(config, null); | ||
| } | ||
|
|
||
| @Inject | ||
| public HdfsDataSegmentPuller( | ||
| @Hdfs final Configuration config, | ||
| @Nullable final ServiceEmitter emitter | ||
| ) | ||
| { | ||
| this.config = config; | ||
| this.emitter = emitter; | ||
| } | ||
|
|
||
| FileUtils.FileCopyResult getSegmentFiles(final Path path, final File outDir) throws SegmentLoadingException | ||
|
|
@@ -235,27 +248,16 @@ FileUtils.FileCopyResult getSegmentFiles(final Path path, final File outDir) thr | |
| catch (Exception e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| } else if (CompressionUtils.isZip(path.getName())) { | ||
| } | ||
|
|
||
| // -------- zip --------- | ||
| // Try to detect format from file extension and decompress | ||
| final CompressionUtils.Format format = CompressionUtils.Format.fromFileName(path.getName()); | ||
| if ((format == CompressionUtils.Format.ZIP || format == CompressionUtils.Format.LZ4)) { | ||
| long startTime = System.currentTimeMillis(); | ||
|
|
||
| final FileUtils.FileCopyResult result = CompressionUtils.unzip( | ||
| new ByteSource() | ||
| { | ||
| @Override | ||
| public InputStream openStream() throws IOException | ||
| { | ||
| return getInputStream(path); | ||
| } | ||
| }, outDir, shouldRetryPredicate(), false | ||
| ); | ||
| final FileUtils.FileCopyResult result = format.decompressDirectory(getInputStream(path), outDir); | ||
|
|
||
| log.info( | ||
| "Unzipped %d bytes from [%s] to [%s]", | ||
| result.size(), | ||
| path.toString(), | ||
| outDir.getAbsolutePath() | ||
| ); | ||
| emitMetrics(format, result.size(), System.currentTimeMillis() - startTime); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Any chance that There is an expectation for metrics to be emitted for non-zip/lz4 too.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. currently only zip is saved in hdfs, gz and dir should be old code path. no need to support it. |
||
|
|
||
| return result; | ||
| } else if (CompressionUtils.isGz(path.getName())) { | ||
|
|
@@ -292,6 +294,17 @@ public InputStream openStream() throws IOException | |
| } | ||
| } | ||
|
|
||
| private void emitMetrics(CompressionUtils.Format format, long size, long duration) | ||
| { | ||
| if (emitter == null) { | ||
| return; | ||
| } | ||
| ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder(); | ||
| metricBuilder.setDimension("format", format); | ||
| emitter.emit(metricBuilder.setMetric("hdfs/pull/size", size)); | ||
| emitter.emit(metricBuilder.setMetric("hdfs/pull/duration", duration)); | ||
GWphua marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| public InputStream getInputStream(Path path) throws IOException | ||
| { | ||
| return buildFileObject(path.toUri(), config).openInputStream(); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Out of scope of this PR, but seems like it may be better to use a factory class instead to handle this logic...
Maybe something to do in the future.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it's better to do refactoring in a separated pr