-
Notifications
You must be signed in to change notification settings - Fork 327
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Smaller lzo indexes: make #43 apply to current codebase
- Loading branch information
Dmitriy Ryaboy
committed
Sep 5, 2014
1 parent
e8c11c2
commit c620b24
Showing
7 changed files
with
609 additions
and
56 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
106 changes: 106 additions & 0 deletions
106
src/main/java/com/hadoop/compression/lzo/LzoBasicIndexSerde.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,106 @@ | ||
/* | ||
* This file is part of Hadoop-Gpl-Compression. | ||
* | ||
* Hadoop-Gpl-Compression is free software: you can redistribute it | ||
* and/or modify it under the terms of the GNU General Public License | ||
* as published by the Free Software Foundation, either version 3 of | ||
* the License, or (at your option) any later version. | ||
* | ||
* Hadoop-Gpl-Compression is distributed in the hope that it will be | ||
* useful, but WITHOUT ANY WARRANTY; without even the implied warranty | ||
* of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the | ||
* GNU General Public License for more details. | ||
* | ||
* You should have received a copy of the GNU General Public License | ||
* along with Hadoop-Gpl-Compression. If not, see | ||
* <http://www.gnu.org/licenses/>. | ||
*/ | ||
|
||
package com.hadoop.compression.lzo; | ||
|
||
|
||
|
||
|
||
import java.io.DataInputStream; | ||
import java.io.DataOutputStream; | ||
import java.io.IOException; | ||
import java.nio.ByteBuffer; | ||
|
||
import org.apache.hadoop.io.DataOutputBuffer; | ||
import org.apache.hadoop.io.IOUtils; | ||
|
||
public class LzoBasicIndexSerde implements LzoIndexSerde { | ||
|
||
private static final int BUFFER_CAPACITY = 16 * 1024 * 8; //size for a 4GB file (with 256KB lzo blocks) | ||
|
||
private DataOutputStream os; | ||
private DataInputStream is; | ||
private ByteBuffer bytesIn; | ||
private long firstLong; | ||
private int numBlocks = 0; | ||
private boolean processedFirstLong = false; | ||
|
||
@Override | ||
public boolean accepts(long firstLong) { | ||
if (firstLong < 0) { | ||
return false; | ||
} else { | ||
this.firstLong = firstLong; | ||
return true; | ||
} | ||
} | ||
|
||
@Override | ||
public void prepareToWrite(DataOutputStream os) throws IOException { | ||
this.os = os; | ||
} | ||
|
||
@Override | ||
public void prepareToRead(DataInputStream is) throws IOException { | ||
this.is = is; | ||
bytesIn = fillBuffer(); | ||
numBlocks = bytesIn.remaining()/8 + 1; // plus one for the first long. | ||
processedFirstLong = false; | ||
} | ||
|
||
@Override | ||
public void writeOffset(long offset) throws IOException { | ||
os.writeLong(offset); | ||
} | ||
|
||
@Override | ||
public void finishWriting() throws IOException { | ||
os.close(); | ||
} | ||
|
||
@Override | ||
public boolean hasNext() throws IOException { | ||
return !processedFirstLong || (bytesIn != null && bytesIn.hasRemaining()); | ||
} | ||
|
||
@Override | ||
public long next() throws IOException { | ||
if (!processedFirstLong) { | ||
processedFirstLong = true; | ||
return firstLong; | ||
} | ||
if (bytesIn != null && bytesIn.hasRemaining()) { | ||
return bytesIn.getLong(); | ||
} else { | ||
throw new IOException("Attempt to read past the edge of the index."); | ||
} | ||
} | ||
|
||
private ByteBuffer fillBuffer() throws IOException { | ||
DataOutputBuffer bytes = new DataOutputBuffer(BUFFER_CAPACITY); | ||
// copy indexIn and close it if finished | ||
IOUtils.copyBytes(is, bytes, 4*1024, true); | ||
return ByteBuffer.wrap(bytes.getData(), 0, bytes.getLength()); | ||
} | ||
|
||
@Override | ||
public void finishReading() throws IOException { | ||
is.close(); | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
67 changes: 67 additions & 0 deletions
67
src/main/java/com/hadoop/compression/lzo/LzoIndexSerde.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,67 @@ | ||
/* | ||
* This file is part of Hadoop-Gpl-Compression. | ||
* | ||
* Hadoop-Gpl-Compression is free software: you can redistribute it | ||
* and/or modify it under the terms of the GNU General Public License | ||
* as published by the Free Software Foundation, either version 3 of | ||
* the License, or (at your option) any later version. | ||
* | ||
* Hadoop-Gpl-Compression is distributed in the hope that it will be | ||
* useful, but WITHOUT ANY WARRANTY; without even the implied warranty | ||
* of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the | ||
* GNU General Public License for more details. | ||
* | ||
* You should have received a copy of the GNU General Public License | ||
* along with Hadoop-Gpl-Compression. If not, see | ||
* <http://www.gnu.org/licenses/>. | ||
*/ | ||
|
||
package com.hadoop.compression.lzo; | ||
|
||
import java.io.DataInputStream; | ||
import java.io.DataOutputStream; | ||
import java.io.IOException; | ||
|
||
public interface LzoIndexSerde { | ||
|
||
/** | ||
* Serdes will be tried in order until one is found that accepts | ||
* the offered format. A format is determined from the first 8 | ||
* bytes (represented as a long) written to the index file. | ||
* <p> | ||
* The first long is somewhat constrained: the topmost bit should be | ||
* 1, the next 31 are a version number by which the appropriate SerDe | ||
* is decided, and the next 32 can have arbitrary data (a header, or | ||
* a length of the header, or an offset.. up to you). | ||
* | ||
* @param firstLong | ||
* @return true if this format is recognized by the SerDe, false otherwise. | ||
*/ | ||
public boolean accepts(long firstLong); | ||
|
||
public void prepareToWrite(DataOutputStream os) throws IOException; | ||
|
||
/** | ||
* Prepare to read the index. Note that the first 8 bits will have been already | ||
* read from this stream, and passed to you in accepts() in the form of a long. | ||
* @param is InputStream to read. | ||
*/ | ||
public void prepareToRead(DataInputStream is) throws IOException; | ||
|
||
/** | ||
* Write the next offset into the file. It is expected that | ||
* the offsets are supplied in order. <code>prepareToWrite()</code> | ||
* should be called before the first invocation of this method. | ||
* @param offset | ||
*/ | ||
public void writeOffset(long offset) throws IOException; | ||
|
||
public void finishWriting() throws IOException; | ||
|
||
public void finishReading() throws IOException; | ||
|
||
public boolean hasNext() throws IOException; | ||
|
||
public long next() throws IOException; | ||
|
||
} |
Oops, something went wrong.