|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +/* |
| 19 | + * The MIT License (MIT) |
| 20 | + * Copyright (c) 2014 Martin Kleppmann |
| 21 | + * |
| 22 | + * Permission is hereby granted, free of charge, to any person obtaining a copy |
| 23 | + * of this software and associated documentation files (the "Software"), to deal |
| 24 | + * in the Software without restriction, including without limitation the rights |
| 25 | + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell |
| 26 | + * copies of the Software, and to permit persons to whom the Software is |
| 27 | + * furnished to do so, subject to the following conditions: |
| 28 | + * |
| 29 | + * The above copyright notice and this permission notice shall be included in |
| 30 | + * all copies or substantial portions of the Software. |
| 31 | + * |
| 32 | + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR |
| 33 | + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, |
| 34 | + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE |
| 35 | + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER |
| 36 | + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, |
| 37 | + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN |
| 38 | + * THE SOFTWARE. |
| 39 | + */ |
| 40 | + |
| 41 | +package org.apache.hadoop.hbase.test.util.warc; |
| 42 | + |
| 43 | +import java.io.BufferedInputStream; |
| 44 | +import java.io.DataInputStream; |
| 45 | +import java.io.FilterInputStream; |
| 46 | +import java.io.IOException; |
| 47 | +import java.io.InputStream; |
| 48 | +import org.apache.hadoop.conf.Configuration; |
| 49 | +import org.apache.hadoop.fs.FileSystem; |
| 50 | +import org.apache.hadoop.fs.Path; |
| 51 | +import org.apache.hadoop.io.compress.CompressionCodec; |
| 52 | +import org.slf4j.Logger; |
| 53 | +import org.slf4j.LoggerFactory; |
| 54 | + |
| 55 | +/** |
| 56 | + * Reads {@link WARCRecord}s from a WARC file, using Hadoop's filesystem APIs. (This means you |
| 57 | + * can read from HDFS, S3 or any other filesystem supported by Hadoop). This implementation is |
| 58 | + * not tied to the MapReduce APIs -- that link is provided by the mapred |
| 59 | + * {@link com.martinkl.warc.mapred.WARCInputFormat} and the mapreduce |
| 60 | + * {@link com.martinkl.warc.mapreduce.WARCInputFormat}. |
| 61 | + */ |
| 62 | +public class WARCFileReader { |
| 63 | + private static final Logger logger = LoggerFactory.getLogger(WARCFileReader.class); |
| 64 | + |
| 65 | + private final long fileSize; |
| 66 | + private CountingInputStream byteStream = null; |
| 67 | + private DataInputStream dataStream = null; |
| 68 | + private long bytesRead = 0, recordsRead = 0; |
| 69 | + |
| 70 | + /** |
| 71 | + * Opens a file for reading. If the filename ends in `.gz`, it is automatically decompressed |
| 72 | + * on the fly. |
| 73 | + * @param conf The Hadoop configuration. |
| 74 | + * @param filePath The Hadoop path to the file that should be read. |
| 75 | + * @throws IOException |
| 76 | + */ |
| 77 | + public WARCFileReader(Configuration conf, Path filePath) throws IOException { |
| 78 | + FileSystem fs = filePath.getFileSystem(conf); |
| 79 | + this.fileSize = fs.getFileStatus(filePath).getLen(); |
| 80 | + logger.info("Reading from " + filePath); |
| 81 | + |
| 82 | + CompressionCodec codec = filePath.getName().endsWith(".gz") ? |
| 83 | + WARCFileWriter.getGzipCodec(conf) : null; |
| 84 | + byteStream = new CountingInputStream(new BufferedInputStream(fs.open(filePath))); |
| 85 | + dataStream = new DataInputStream(codec == null ? byteStream : codec.createInputStream(byteStream)); |
| 86 | + } |
| 87 | + |
| 88 | + /** |
| 89 | + * Reads the next record from the file. |
| 90 | + * @return The record that was read. |
| 91 | + * @throws IOException |
| 92 | + */ |
| 93 | + public WARCRecord read() throws IOException { |
| 94 | + WARCRecord record = new WARCRecord(dataStream); |
| 95 | + recordsRead++; |
| 96 | + return record; |
| 97 | + } |
| 98 | + |
| 99 | + /** |
| 100 | + * Closes the file. No more reading is possible after the file has been closed. |
| 101 | + * @throws IOException |
| 102 | + */ |
| 103 | + public void close() throws IOException { |
| 104 | + if (dataStream != null) dataStream.close(); |
| 105 | + byteStream = null; |
| 106 | + dataStream = null; |
| 107 | + } |
| 108 | + |
| 109 | + /** |
| 110 | + * Returns the number of records that have been read since the file was opened. |
| 111 | + */ |
| 112 | + public long getRecordsRead() { |
| 113 | + return recordsRead; |
| 114 | + } |
| 115 | + |
| 116 | + /** |
| 117 | + * Returns the number of bytes that have been read from file since it was opened. |
| 118 | + * If the file is compressed, this refers to the compressed file size. |
| 119 | + */ |
| 120 | + public long getBytesRead() { |
| 121 | + return bytesRead; |
| 122 | + } |
| 123 | + |
| 124 | + /** |
| 125 | + * Returns the proportion of the file that has been read, as a number between 0.0 |
| 126 | + * and 1.0. |
| 127 | + */ |
| 128 | + public float getProgress() { |
| 129 | + if (fileSize == 0) return 1.0f; |
| 130 | + return (float) bytesRead / (float) fileSize; |
| 131 | + } |
| 132 | + |
| 133 | + private class CountingInputStream extends FilterInputStream { |
| 134 | + public CountingInputStream(InputStream in) { |
| 135 | + super(in); |
| 136 | + } |
| 137 | + |
| 138 | + @Override |
| 139 | + public int read() throws IOException { |
| 140 | + int result = in.read(); |
| 141 | + if (result != -1) bytesRead++; |
| 142 | + return result; |
| 143 | + } |
| 144 | + |
| 145 | + @Override |
| 146 | + public int read(byte[] b, int off, int len) throws IOException { |
| 147 | + int result = in.read(b, off, len); |
| 148 | + if (result != -1) bytesRead += result; |
| 149 | + return result; |
| 150 | + } |
| 151 | + |
| 152 | + @Override |
| 153 | + public long skip(long n) throws IOException { |
| 154 | + long result = in.skip(n); |
| 155 | + bytesRead += result; |
| 156 | + return result; |
| 157 | + } |
| 158 | + } |
| 159 | +} |
0 commit comments