HBASE-61 Create an HBase-specific MapFile implementation

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@747672 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-02-25 05:59:26 +00:00
parent be84747ad3
commit f073189447
28 changed files with 7528 additions and 3175 deletions

View File

@ -323,6 +323,22 @@
if you have small cells and want faster random-access of individual cells.
</description>
</property>
<property>
<name>hfile.min.blocksize.size</name>
<value>65536</value>
<description>Minimum store file block size. The smaller you make this, the
bigger your index and the less you fetch on a random-access. Set size down
if you have small cells and want faster random-access of individual cells.
</description>
</property>
<property>
<name>hfile.min.blocksize.size</name>
<value>65536</value>
<description>Minimum store file block size. The smaller you make this, the
bigger your index and the less you fetch on a random-access. Set size down
if you have small cells and want faster random-access of individual cells.
</description>
</property>
<property>
<name>hbase.io.seqfile.compression.type</name>
<value>NONE</value>

View File

@ -0,0 +1,198 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.io;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.util.Bytes;
/**
* A facade for a {@link org.apache.hadoop.io.hfile.HFile.Reader} that serves up
* either the top or bottom half of a HFile where 'bottom' is the first half
* of the file containing the keys that sort lowest and 'top' is the second half
* of the file with keys that sort greater than those of the bottom half.
* The top includes the split files midkey, of the key that follows if it does
* not exist in the file.
*
* <p>This type works in tandem with the {@link Reference} type. This class
* is used reading while Reference is used writing.
*
* <p>This file is not splitable. Calls to {@link #midKey()} return null.
*/
public class HalfHFileReader extends HFile.Reader {
static final Log LOG = LogFactory.getLog(HalfHFileReader.class);
private final boolean top;
// This is the key we split around. Its the first possible entry on a row:
// i.e. empty column and a timestamp of LATEST_TIMESTAMP.
private final byte [] splitkey;
/**
* @param fs
* @param f
* @param c
* @param r
* @throws IOException
*/
public HalfHFileReader(final FileSystem fs, final Path p, final BlockCache c,
final Reference r)
throws IOException {
super(fs, p, c);
// This is not actual midkey for this half-file; its just border
// around which we split top and bottom. Have to look in files to find
// actual last and first keys for bottom and top halves. Half-files don't
// have an actual midkey themselves. No midkey is how we indicate file is
// not splittable.
this.splitkey = r.getSplitKey();
// Is it top or bottom half?
this.top = Reference.isTopFileRegion(r.getFileRegion());
}
public HFileScanner getScanner() {
final HFileScanner s = super.getScanner();
return new HFileScanner() {
final HFileScanner delegate = s;
public ByteBuffer getKey() {
return delegate.getKey();
}
public String getKeyString() {
return delegate.getKeyString();
}
public ByteBuffer getValue() {
return delegate.getValue();
}
public String getValueString() {
return delegate.getValueString();
}
public boolean next() throws IOException {
boolean b = delegate.next();
if (!b) {
return b;
}
if (!top) {
ByteBuffer bb = getKey();
if (getComparator().compare(bb.array(), bb.arrayOffset(), bb.limit(),
splitkey, 0, splitkey.length) >= 0) {
return false;
}
}
return true;
}
public boolean seekBefore(byte[] key) throws IOException {
if (top) {
if (getComparator().compare(key, splitkey) < 0) {
return false;
}
} else {
if (getComparator().compare(key, splitkey) >= 0) {
return seekBefore(splitkey);
}
}
return this.delegate.seekBefore(key);
}
public boolean seekTo() throws IOException {
if (top) {
int r = this.delegate.seekTo(splitkey);
if (r < 0) {
// midkey is < first key in file
return this.delegate.seekTo();
}
if (r > 0) {
return this.delegate.next();
}
return true;
}
boolean b = delegate.seekTo();
if (!b) {
return b;
}
// Check key.
ByteBuffer k = this.delegate.getKey();
return this.delegate.getReader().getComparator().
compare(k.array(), k.arrayOffset(), k.limit(),
splitkey, 0, splitkey.length) < 0;
}
public int seekTo(byte[] key) throws IOException {
if (top) {
if (getComparator().compare(key, splitkey) < 0) {
return -1;
}
} else {
if (getComparator().compare(key, splitkey) >= 0) {
// we would place the scanner in the second half.
// it might be an error to return false here ever...
boolean res = delegate.seekBefore(splitkey);
if (!res) {
throw new IOException("Seeking for a key in bottom of file, but key exists in top of file, failed on seekBefore(midkey)");
}
return 1;
}
}
return delegate.seekTo(key);
}
public Reader getReader() {
return this.delegate.getReader();
}
public boolean isSeeked() {
return this.delegate.isSeeked();
}
};
}
public byte[] getLastKey() {
if (top) {
return super.getLastKey();
} else {
HFileScanner scanner = getScanner();
try {
if (scanner.seekBefore(this.splitkey)) {
return Bytes.toBytes(scanner.getKey());
}
} catch (IOException e) {
LOG.warn("Failed seekBefore " + Bytes.toString(this.splitkey), e);
}
return null;
}
}
public byte[] midkey() throws IOException {
// Returns null to indicate file is not splitable.
return null;
}
}

View File

@ -0,0 +1,42 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.nio.ByteBuffer;
/**
* Block cache interface.
* TODO: Add filename or hash of filename to block cache key.
*/
public interface BlockCache {
/**
* Add block to cache.
* @param blockNumber Zero-based file block number.
* @param buf The block contents wrapped in a ByteBuffer.
*/
public void cacheBlock(String blockName, ByteBuffer buf);
/**
* Fetch block from cache.
* @param blockNumber Block number to fetch.
* @return Block or null if block is not in the cache.
*/
public ByteBuffer getBlock(String blockName);
}

View File

@ -0,0 +1,143 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.io.InputStream;
import org.apache.hadoop.fs.FSDataInputStream;
/**
* BoundedRangeFIleInputStream abstracts a contiguous region of a Hadoop
* FSDataInputStream as a regular input stream. One can create multiple
* BoundedRangeFileInputStream on top of the same FSDataInputStream and they
* would not interfere with each other.
* Copied from hadoop-335 tfile.
*/
class BoundedRangeFileInputStream extends InputStream {
private FSDataInputStream in;
private long pos;
private long end;
private long mark;
private final byte[] oneByte = new byte[1];
/**
* Constructor
*
* @param in
* The FSDataInputStream we connect to.
* @param offset
* Beginning offset of the region.
* @param length
* Length of the region.
*
* The actual length of the region may be smaller if (off_begin +
* length) goes beyond the end of FS input stream.
*/
public BoundedRangeFileInputStream(FSDataInputStream in, long offset,
long length) {
if (offset < 0 || length < 0) {
throw new IndexOutOfBoundsException("Invalid offset/length: " + offset
+ "/" + length);
}
this.in = in;
this.pos = offset;
this.end = offset + length;
this.mark = -1;
}
@Override
public int available() throws IOException {
int avail = in.available();
if (pos + avail > end) {
avail = (int) (end - pos);
}
return avail;
}
@Override
public int read() throws IOException {
int ret = read(oneByte);
if (ret == 1) return oneByte[0] & 0xff;
return -1;
}
@Override
public int read(byte[] b) throws IOException {
return read(b, 0, b.length);
}
@Override
public int read(byte[] b, int off, int len) throws IOException {
if ((off | len | (off + len) | (b.length - (off + len))) < 0) {
throw new IndexOutOfBoundsException();
}
int n = (int) Math.min(Integer.MAX_VALUE, Math.min(len, (end - pos)));
if (n == 0) return -1;
int ret = 0;
synchronized (in) {
in.seek(pos);
ret = in.read(b, off, n);
}
// / ret = in.read(pos, b, off, n);
if (ret < 0) {
end = pos;
return -1;
}
pos += ret;
return ret;
}
@Override
/*
* We may skip beyond the end of the file.
*/
public long skip(long n) throws IOException {
long len = Math.min(n, end - pos);
pos += len;
return len;
}
@Override
public void mark(int readlimit) {
mark = pos;
}
@Override
public void reset() throws IOException {
if (mark < 0) throw new IOException("Resetting to invalid mark");
pos = mark;
}
@Override
public boolean markSupported() {
return true;
}
@Override
public void close() {
// Invalidate the state of the stream.
in = null;
pos = end;
mark = -1;
}
}

View File

@ -0,0 +1,324 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.FilterOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.compress.CodecPool;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionInputStream;
import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
import org.apache.hadoop.io.compress.Decompressor;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.io.compress.LzoCodec;
/**
* Compression related stuff.
* Copied from hadoop-3315 tfile.
*/
public final class Compression {
static final Log LOG = LogFactory.getLog(Compression.class);
/**
* Prevent the instantiation of class.
*/
private Compression() {
super();
}
static class FinishOnFlushCompressionStream extends FilterOutputStream {
public FinishOnFlushCompressionStream(CompressionOutputStream cout) {
super(cout);
}
@Override
public void write(byte b[], int off, int len) throws IOException {
out.write(b, off, len);
}
@Override
public void flush() throws IOException {
CompressionOutputStream cout = (CompressionOutputStream) out;
cout.finish();
cout.flush();
cout.resetState();
}
}
/**
* Compression algorithms.
*/
public static enum Algorithm {
LZO("lzo") {
private LzoCodec codec;
@Override
CompressionCodec getCodec() {
if (codec == null) {
Configuration conf = new Configuration();
conf.setBoolean("hadoop.native.lib", true);
codec = new LzoCodec();
codec.setConf(conf);
}
return codec;
}
@Override
public synchronized InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException {
InputStream bis1 = null;
if (downStreamBufferSize > 0) {
bis1 = new BufferedInputStream(downStream, downStreamBufferSize);
}
else {
bis1 = downStream;
}
codec.getConf()
.setInt("io.compression.codec.lzo.buffersize", 64 * 1024);
CompressionInputStream cis =
codec.createInputStream(bis1, decompressor);
BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE);
return bis2;
}
@Override
public synchronized OutputStream createCompressionStream(
OutputStream downStream, Compressor compressor,
int downStreamBufferSize) throws IOException {
OutputStream bos1 = null;
if (downStreamBufferSize > 0) {
bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
}
else {
bos1 = downStream;
}
codec.getConf()
.setInt("io.compression.codec.lzo.buffersize", 64 * 1024);
CompressionOutputStream cos =
codec.createOutputStream(bos1, compressor);
BufferedOutputStream bos2 =
new BufferedOutputStream(new FinishOnFlushCompressionStream(cos),
DATA_OBUF_SIZE);
return bos2;
}
},
GZ("gz") {
private GzipCodec codec;
@Override
CompressionCodec getCodec() {
if (codec == null) {
Configuration conf = new Configuration();
conf.setBoolean("hadoop.native.lib", true);
codec = new GzipCodec();
codec.setConf(conf);
}
return codec;
}
@Override
public synchronized InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException {
// Set the internal buffer size to read from down stream.
if (downStreamBufferSize > 0) {
codec.getConf().setInt("io.file.buffer.size", downStreamBufferSize);
}
CompressionInputStream cis =
codec.createInputStream(downStream, decompressor);
BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE);
return bis2;
}
@Override
public synchronized OutputStream createCompressionStream(
OutputStream downStream, Compressor compressor,
int downStreamBufferSize) throws IOException {
OutputStream bos1 = null;
if (downStreamBufferSize > 0) {
bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
}
else {
bos1 = downStream;
}
codec.getConf().setInt("io.file.buffer.size", 32 * 1024);
CompressionOutputStream cos =
codec.createOutputStream(bos1, compressor);
BufferedOutputStream bos2 =
new BufferedOutputStream(new FinishOnFlushCompressionStream(cos),
DATA_OBUF_SIZE);
return bos2;
}
},
NONE("none") {
@Override
CompressionCodec getCodec() {
return null;
}
@Override
public synchronized InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException {
if (downStreamBufferSize > 0) {
return new BufferedInputStream(downStream, downStreamBufferSize);
}
// else {
// Make sure we bypass FSInputChecker buffer.
// return new BufferedInputStream(downStream, 1024);
// }
// }
return downStream;
}
@Override
public synchronized OutputStream createCompressionStream(
OutputStream downStream, Compressor compressor,
int downStreamBufferSize) throws IOException {
if (downStreamBufferSize > 0) {
return new BufferedOutputStream(downStream, downStreamBufferSize);
}
return downStream;
}
};
private final String compressName;
// data input buffer size to absorb small reads from application.
private static final int DATA_IBUF_SIZE = 1 * 1024;
// data output buffer size to absorb small writes from application.
private static final int DATA_OBUF_SIZE = 4 * 1024;
Algorithm(String name) {
this.compressName = name;
}
abstract CompressionCodec getCodec();
public abstract InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException;
public abstract OutputStream createCompressionStream(
OutputStream downStream, Compressor compressor, int downStreamBufferSize)
throws IOException;
public Compressor getCompressor() {
CompressionCodec codec = getCodec();
if (codec != null) {
Compressor compressor = CodecPool.getCompressor(codec);
if (compressor != null) {
if (compressor.finished()) {
// Somebody returns the compressor to CodecPool but is still using
// it.
LOG
.warn("Compressor obtained from CodecPool is already finished()");
// throw new AssertionError(
// "Compressor obtained from CodecPool is already finished()");
}
else {
LOG.debug("Got a compressor: " + compressor.hashCode());
}
compressor.reset();
}
return compressor;
}
return null;
}
public void returnCompressor(Compressor compressor) {
if (compressor != null) {
LOG.debug("Return a compressor: " + compressor.hashCode());
CodecPool.returnCompressor(compressor);
}
}
public Decompressor getDecompressor() {
CompressionCodec codec = getCodec();
if (codec != null) {
Decompressor decompressor = CodecPool.getDecompressor(codec);
if (decompressor != null) {
if (decompressor.finished()) {
// Somebody returns the decompressor to CodecPool but is still using
// it.
LOG
.warn("Deompressor obtained from CodecPool is already finished()");
// throw new AssertionError(
// "Decompressor obtained from CodecPool is already finished()");
}
else {
LOG.debug("Got a decompressor: " + decompressor.hashCode());
}
decompressor.reset();
}
return decompressor;
}
return null;
}
public void returnDecompressor(Decompressor decompressor) {
if (decompressor != null) {
LOG.debug("Returned a decompressor: " + decompressor.hashCode());
CodecPool.returnDecompressor(decompressor);
}
}
public String getName() {
return compressName;
}
}
static Algorithm getCompressionAlgorithmByName(String compressName) {
Algorithm[] algos = Algorithm.class.getEnumConstants();
for (Algorithm a : algos) {
if (a.getName().equals(compressName)) {
return a;
}
}
throw new IllegalArgumentException(
"Unsupported compression algorithm name: " + compressName);
}
static String[] getSupportedAlgorithms() {
Algorithm[] algos = Algorithm.class.getEnumConstants();
String[] ret = new String[algos.length];
int i = 0;
for (Algorithm a : algos) {
ret[i++] = a.getName();
}
return ret;
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,112 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.nio.ByteBuffer;
/**
* A scanner allows you to position yourself within a HFile and
* scan through it. It allows you to reposition yourself as well.
*
* <p>A scanner doesn't always have a key/value that it is pointing to
* when it is first created and before
* {@link #seekTo()}/{@link #seekTo(byte[])} are called.
* In this case, {@link #getKey()}/{@link #getValue()} returns null. At most
* other times, a key and value will be available. The general pattern is that
* you position the Scanner using the seekTo variants and then getKey and
* getValue.
*/
public interface HFileScanner {
/**
* SeekTo or just before the passed <code>key</code>. Examine the return
* code to figure whether we found the key or not.
* Consider the key stream of all the keys in the file,
* <code>k[0] .. k[n]</code>, where there are n keys in the file.
* @param key Key to find.
* @return -1, if key < k[0], no position;
* 0, such that k[i] = key and scanner is left in position i; and
* 1, such that k[i] < key, and scanner is left in position i.
* Furthermore, there may be a k[i+1], such that k[i] < key < k[i+1]
* but there may not be a k[i+1], and next() will return false (EOF).
* @throws IOException
*/
public int seekTo(byte[] key) throws IOException;
/**
* Consider the key stream of all the keys in the file,
* <code>k[0] .. k[n]</code>, where there are n keys in the file.
* @param key Key to find
* @return false if key <= k[0] or true with scanner in position 'i' such
* that: k[i] < key. Furthermore: there may be a k[i+1], such that
* k[i] < key <= k[i+1] but there may also NOT be a k[i+1], and next() will
* return false (EOF).
*/
public boolean seekBefore(byte [] key) throws IOException;
/**
* Positions this scanner at the start of the file.
* @return False if empty file; i.e. a call to next would return false and
* the current key and value are undefined.
* @throws IOException
*/
public boolean seekTo() throws IOException;
/**
* Scans to the next entry in the file.
* @return Returns false if you are at the end otherwise true if more in file.
* @throws IOException
*/
public boolean next() throws IOException;
/**
* Gets a buffer view to the current key. You must call
* {@link #seekTo(byte[])} before this method.
* @return byte buffer for the key. The limit is set to the key size, and the
* position is 0, the start of the buffer view.
*/
public ByteBuffer getKey();
/**
* Gets a buffer view to the current value. You must call
* {@link #seekTo(byte[])} before this method.
*
* @return byte buffer for the value. The limit is set to the value size, and
* the position is 0, the start of the buffer view.
*/
public ByteBuffer getValue();
/**
* Convenience method to get a copy of the key as a string - interpreting the
* bytes as UTF8. You must call {@link #seekTo(byte[])} before this method.
* @return key as a string
*/
public String getKeyString();
/**
* Convenience method to get a copy of the value as a string - interpreting
* the bytes as UTF8. You must call {@link #seekTo(byte[])} before this method.
* @return value as a string
*/
public String getValueString();
/**
* @return Reader that underlies this Scanner instance.
*/
public HFile.Reader getReader();
/**
* @return True is scanner has had one of the seek calls invoked; i.e.
* {@link #seekBefore(byte[])} or {@link #seekTo()} or {@link #seekTo(byte[])}.
* Otherwise returns false.
*/
public boolean isSeeked();
}

View File

@ -0,0 +1,56 @@
package org.apache.hadoop.hbase.io.hfile;
import java.lang.ref.ReferenceQueue;
import java.lang.ref.SoftReference;
import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.Map;
/**
* Simple one RFile soft reference cache.
*/
public class SimpleBlockCache implements BlockCache {
private static class Ref extends SoftReference<ByteBuffer> {
public String blockId;
public Ref(String blockId, ByteBuffer buf, ReferenceQueue q) {
super(buf, q);
this.blockId = blockId;
}
}
private Map<String,Ref> cache =
new HashMap<String,Ref>();
private ReferenceQueue q = new ReferenceQueue();
public int dumps = 0;
public SimpleBlockCache() {
super();
}
void processQueue() {
Ref r;
while ( (r = (Ref)q.poll()) != null) {
cache.remove(r.blockId);
dumps++;
}
}
public synchronized int size() {
processQueue();
return cache.size();
}
@Override
public synchronized ByteBuffer getBlock(String blockName) {
processQueue(); // clear out some crap.
Ref ref = cache.get(blockName);
if (ref == null)
return null;
return ref.get();
}
@Override
public synchronized void cacheBlock(String blockName, ByteBuffer buf) {
cache.put(blockName, new Ref(blockName, buf, q));
}
}

View File

@ -0,0 +1,25 @@
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
<html>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<head />
<body bgcolor="white">
Provides the hbase data+index+metadata file.
</body>
</html>

File diff suppressed because it is too large Load Diff

View File

@ -1,558 +0,0 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.DataInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.io.BloomFilterMapFile;
import org.apache.hadoop.hbase.io.HalfMapFileReader;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.io.MapFile;
import org.apache.hadoop.hbase.io.SequenceFile;
/**
* A HStore data file. HStores usually have one or more of these files. They
* are produced by flushing the memcache to disk.
*
* <p>Each HStore maintains a bunch of different data files. The filename is a
* mix of the parent dir, the region name, the column name, and a file
* identifier. The name may also be a reference to a store file located
* elsewhere. This class handles all that path-building stuff for you.
*
* <p>An HStoreFile usually tracks 4 things: its parent dir, the region
* identifier, the column family, and the file identifier. If you know those
* four things, you know how to obtain the right HStoreFile. HStoreFiles may
* also reference store files in another region serving either from
* the top-half of the remote file or from the bottom-half. Such references
* are made fast splitting regions.
*
* <p>Plain HStoreFiles are named for a randomly generated id as in:
* <code>1278437856009925445</code> A file by this name is made in both the
* <code>mapfiles</code> and <code>info</code> subdirectories of a
* HStore columnfamily directoy: E.g. If the column family is 'anchor:', then
* under the region directory there is a subdirectory named 'anchor' within
* which is a 'mapfiles' and 'info' subdirectory. In each will be found a
* file named something like <code>1278437856009925445</code>, one to hold the
* data in 'mapfiles' and one under 'info' that holds the sequence id for this
* store file.
*
* <p>References to store files located over in some other region look like
* this:
* <code>1278437856009925445.hbaserepository,qAReLZD-OyQORZWq_vqR1k==,959247014679548184</code>:
* i.e. an id followed by the name of the referenced region. The data
* ('mapfiles') of HStoreFile references are empty. The accompanying
* <code>info</code> file contains the
* midkey, the id of the remote store we're referencing and whether we're
* to serve the top or bottom region of the remote store file. Note, a region
* is not splitable if it has instances of store file references (References
* are cleaned up by compactions).
*
* <p>When merging or splitting HRegions, we might want to modify one of the
* params for an HStoreFile (effectively moving it elsewhere).
*/
public class HStoreFile implements HConstants {
static final Log LOG = LogFactory.getLog(HStoreFile.class.getName());
static final byte INFO_SEQ_NUM = 0;
static final byte MAJOR_COMPACTION = INFO_SEQ_NUM + 1;
static final String HSTORE_DATFILE_DIR = "mapfiles";
static final String HSTORE_INFO_DIR = "info";
static final String HSTORE_FILTER_DIR = "filter";
private final static Random rand = new Random();
private final Path basedir;
private final int encodedRegionName;
private final byte [] colFamily;
private final long fileId;
private final HBaseConfiguration conf;
private final FileSystem fs;
private final Reference reference;
private final HRegionInfo hri;
/* If true, this file was product of a major compaction.
*/
private boolean majorCompaction = false;
private long indexLength;
/**
* Constructor that fully initializes the object
* @param conf Configuration object
* @param basedir qualified path that is parent of region directory
* @param colFamily name of the column family
* @param fileId file identifier
* @param ref Reference to another HStoreFile.
* @param hri The region info for this file (HACK HBASE-868). TODO: Fix.
* @throws IOException
*/
HStoreFile(HBaseConfiguration conf, FileSystem fs, Path basedir,
final HRegionInfo hri, byte [] colFamily, long fileId,
final Reference ref)
throws IOException {
this(conf, fs, basedir, hri, colFamily, fileId, ref, false);
}
/**
* Constructor that fully initializes the object
* @param conf Configuration object
* @param basedir qualified path that is parent of region directory
* @param colFamily name of the column family
* @param fileId file identifier
* @param ref Reference to another HStoreFile.
* @param hri The region info for this file (HACK HBASE-868). TODO: Fix.
* @param mc Try if this file was result of a major compression.
* @throws IOException
*/
HStoreFile(HBaseConfiguration conf, FileSystem fs, Path basedir,
final HRegionInfo hri, byte [] colFamily, long fileId,
final Reference ref, final boolean mc)
throws IOException {
this.conf = conf;
this.fs = fs;
this.basedir = basedir;
this.encodedRegionName = hri.getEncodedName();
this.colFamily = colFamily;
this.hri = hri;
long id = fileId;
if (id == -1) {
Path mapdir = HStoreFile.getMapDir(basedir, encodedRegionName, colFamily);
Path testpath = null;
do {
id = Math.abs(rand.nextLong());
testpath = new Path(mapdir, createHStoreFilename(id, -1));
} while(fs.exists(testpath));
}
this.fileId = id;
// If a reference, construction does not write the pointer files. Thats
// done by invocations of writeReferenceFiles(hsf, fs). Happens at split.
this.reference = ref;
this.majorCompaction = mc;
}
/** @return the region name */
boolean isReference() {
return reference != null;
}
Reference getReference() {
return reference;
}
int getEncodedRegionName() {
return this.encodedRegionName;
}
/** @return the column family */
byte [] getColFamily() {
return colFamily;
}
/** @return the file identifier */
long getFileId() {
return fileId;
}
// Build full filenames from those components
/** @return path for MapFile */
Path getMapFilePath() {
if (isReference()) {
return getMapFilePath(encodedRegionName, fileId,
reference.getEncodedRegionName());
}
return getMapFilePath(this.encodedRegionName, fileId);
}
private Path getMapFilePath(final Reference r) {
if (r == null) {
return getMapFilePath();
}
return getMapFilePath(r.getEncodedRegionName(), r.getFileId());
}
private Path getMapFilePath(final int encodedName, final long fid) {
return getMapFilePath(encodedName, fid, HRegionInfo.NO_HASH);
}
private Path getMapFilePath(final int encodedName, final long fid,
final int ern) {
return new Path(HStoreFile.getMapDir(basedir, encodedName, colFamily),
createHStoreFilename(fid, ern));
}
/** @return path for info file */
Path getInfoFilePath() {
if (isReference()) {
return getInfoFilePath(encodedRegionName, fileId,
reference.getEncodedRegionName());
}
return getInfoFilePath(encodedRegionName, fileId);
}
private Path getInfoFilePath(final int encodedName, final long fid) {
return getInfoFilePath(encodedName, fid, HRegionInfo.NO_HASH);
}
private Path getInfoFilePath(final int encodedName, final long fid,
final int ern) {
return new Path(HStoreFile.getInfoDir(basedir, encodedName, colFamily),
createHStoreFilename(fid, ern));
}
// File handling
/*
* Split by making two new store files that reference top and bottom regions
* of original store file.
* @param midKey
* @param dstA
* @param dstB
* @param fs
* @param c
* @throws IOException
*
* @param midKey the key which will be the starting key of the second region
* @param dstA the file which will contain keys from the start of the source
* @param dstB the file which will contain keys from midKey to end of source
* @param fs file system
* @param c configuration
* @throws IOException
*/
void splitStoreFile(final HStoreFile dstA, final HStoreFile dstB,
final FileSystem fs)
throws IOException {
dstA.writeReferenceFiles(fs);
dstB.writeReferenceFiles(fs);
}
void writeReferenceFiles(final FileSystem fs)
throws IOException {
createOrFail(fs, getMapFilePath());
writeSplitInfo(fs);
}
/*
* If reference, create and write the remote store file id, the midkey and
* whether we're going against the top file region of the referent out to
* the info file.
* @param p Path to info file.
* @param hsf
* @param fs
* @throws IOException
*/
private void writeSplitInfo(final FileSystem fs) throws IOException {
Path p = getInfoFilePath();
if (fs.exists(p)) {
throw new IOException("File already exists " + p.toString());
}
FSDataOutputStream out = fs.create(p);
try {
reference.write(out);
} finally {
out.close();
}
}
/**
* @see #writeSplitInfo(FileSystem fs)
*/
static Reference readSplitInfo(final Path p, final FileSystem fs)
throws IOException {
FSDataInputStream in = fs.open(p);
try {
Reference r = new Reference();
r.readFields(in);
return r;
} finally {
in.close();
}
}
private void createOrFail(final FileSystem fs, final Path p)
throws IOException {
if (fs.exists(p)) {
throw new IOException("File already exists " + p.toString());
}
if (!fs.createNewFile(p)) {
throw new IOException("Failed create of " + p);
}
}
/**
* Reads in an info file
*
* @param filesystem file system
* @return The sequence id contained in the info file
* @throws IOException
*/
long loadInfo(final FileSystem filesystem) throws IOException {
Path p = null;
if (isReference()) {
p = getInfoFilePath(reference.getEncodedRegionName(),
this.reference.getFileId());
} else {
p = getInfoFilePath();
}
long length = filesystem.getFileStatus(p).getLen();
boolean hasMoreThanSeqNum = length > (Byte.SIZE + Bytes.SIZEOF_LONG);
DataInputStream in = new DataInputStream(filesystem.open(p));
try {
byte flag = in.readByte();
if (flag == INFO_SEQ_NUM) {
if (hasMoreThanSeqNum) {
flag = in.readByte();
if (flag == MAJOR_COMPACTION) {
this.majorCompaction = in.readBoolean();
}
}
return in.readLong();
}
throw new IOException("Cannot process log file: " + p);
} finally {
in.close();
}
}
/**
* Writes the file-identifier to disk
*
* @param filesystem file system
* @param infonum file id
* @throws IOException
*/
void writeInfo(final FileSystem filesystem, final long infonum)
throws IOException {
writeInfo(filesystem, infonum, false);
}
/**
* Writes the file-identifier to disk
*
* @param filesystem file system
* @param infonum file id
* @param mc True if this file is product of a major compaction
* @throws IOException
*/
void writeInfo(final FileSystem filesystem, final long infonum,
final boolean mc)
throws IOException {
Path p = getInfoFilePath();
FSDataOutputStream out = filesystem.create(p);
try {
out.writeByte(INFO_SEQ_NUM);
out.writeLong(infonum);
if (mc) {
// Set whether major compaction flag on this file.
this.majorCompaction = mc;
out.writeByte(MAJOR_COMPACTION);
out.writeBoolean(mc);
}
} finally {
out.close();
}
}
/**
* Delete store map files.
* @throws IOException
*/
public void delete() throws IOException {
fs.delete(getMapFilePath(), true);
fs.delete(getInfoFilePath(), true);
}
/**
* Renames the mapfiles and info directories under the passed
* <code>hsf</code> directory.
* @param fs
* @param hsf
* @return True if succeeded.
* @throws IOException
*/
public boolean rename(final FileSystem fs, final HStoreFile hsf)
throws IOException {
Path src = getMapFilePath();
if (!fs.exists(src)) {
throw new FileNotFoundException(src.toString());
}
boolean success = fs.rename(src, hsf.getMapFilePath());
if (!success) {
LOG.warn("Failed rename of " + src + " to " + hsf.getMapFilePath());
} else {
src = getInfoFilePath();
if (!fs.exists(src)) {
throw new FileNotFoundException(src.toString());
}
success = fs.rename(src, hsf.getInfoFilePath());
if (!success) {
LOG.warn("Failed rename of " + src + " to " + hsf.getInfoFilePath());
}
}
return success;
}
/**
* Get reader for the store file map file.
* Client is responsible for closing file when done.
* @param fs
* @param bloomFilter If true, a bloom filter exists
* @param blockCacheEnabled If true, MapFile blocks should be cached.
* @return BloomFilterMapFile.Reader
* @throws IOException
*/
public synchronized BloomFilterMapFile.Reader getReader(final FileSystem fs,
final boolean bloomFilter, final boolean blockCacheEnabled)
throws IOException {
if (isReference()) {
return new HalfMapFileReader(fs,
getMapFilePath(reference).toString(), conf,
reference.getFileRegion(), reference.getMidkey(), bloomFilter,
blockCacheEnabled, this.hri);
}
return new BloomFilterMapFile.Reader(fs, getMapFilePath().toString(),
conf, bloomFilter, blockCacheEnabled, this.hri);
}
/**
* Get a store file writer.
* Client is responsible for closing file when done.
* @param fs
* @param compression Pass <code>SequenceFile.CompressionType.NONE</code>
* for none.
* @param bloomFilter If true, create a bloom filter
* @param nrows number of rows expected. Required if bloomFilter is true.
* @return MapFile.Writer
* @throws IOException
*/
public MapFile.Writer getWriter(final FileSystem fs,
final SequenceFile.CompressionType compression,
final boolean bloomFilter, int nrows)
throws IOException {
if (isReference()) {
throw new IOException("Illegal Access: Cannot get a writer on a" +
"HStoreFile reference");
}
return new BloomFilterMapFile.Writer(conf, fs,
getMapFilePath().toString(), compression, bloomFilter, nrows, this.hri);
}
/**
* @return Length of the store map file. If a reference, size is
* approximation.
* @throws IOException
*/
public long length() throws IOException {
Path p = new Path(getMapFilePath(reference), MapFile.DATA_FILE_NAME);
long l = p.getFileSystem(conf).getFileStatus(p).getLen();
return (isReference())? l / 2: l;
}
/**
* @return Length of the store map file index.
* @throws IOException
*/
public synchronized long indexLength() throws IOException {
if (indexLength == 0) {
Path p = new Path(getMapFilePath(reference), MapFile.INDEX_FILE_NAME);
indexLength = p.getFileSystem(conf).getFileStatus(p).getLen();
}
return indexLength;
}
@Override
public String toString() {
return encodedRegionName + "/" + Bytes.toString(colFamily) + "/" + fileId +
(isReference()? "-" + reference.toString(): "");
}
/**
* @return True if this file was made by a major compaction.
*/
public boolean isMajorCompaction() {
return this.majorCompaction;
}
private static String createHStoreFilename(final long fid,
final int encodedRegionName) {
return Long.toString(fid) +
((encodedRegionName != HRegionInfo.NO_HASH)?
"." + encodedRegionName : "");
}
/**
* @param dir Base directory
* @param encodedRegionName Encoding of region name.
* @param f Column family.
* @return path for map file directory
*/
public static Path getMapDir(Path dir, int encodedRegionName,
final byte [] f) {
return getFamilySubDir(dir, encodedRegionName, f, HSTORE_DATFILE_DIR);
}
/**
* @param dir Base directory
* @param encodedRegionName Encoding of region name.
* @param f Column family.
* @return the info directory path
*/
public static Path getInfoDir(Path dir, int encodedRegionName, byte [] f) {
return getFamilySubDir(dir, encodedRegionName, f, HSTORE_INFO_DIR);
}
/**
* @param dir Base directory
* @param encodedRegionName Encoding of region name.
* @param f Column family.
* @return the bloom filter directory path
*/
@Deprecated
public static Path getFilterDir(Path dir, int encodedRegionName,
final byte [] f) {
return getFamilySubDir(dir, encodedRegionName, f, HSTORE_FILTER_DIR);
}
/*
* @param base Base directory
* @param encodedRegionName Encoding of region name.
* @param f Column family.
* @param subdir Subdirectory to create under column family/store directory.
* @return
*/
private static Path getFamilySubDir(final Path base,
final int encodedRegionName, final byte [] f, final String subdir) {
return new Path(base, new Path(Integer.toString(encodedRegionName),
new Path(Bytes.toString(f), subdir)));
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,443 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.io.HalfHFileReader;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.RawComparator;
/**
* A Store data file. Stores usually have one or more of these files. They
* are produced by flushing the memcache to disk. To
* create, call {@link #getWriter(FileSystem, Path)} and append data. Be
* sure to add any metadata before calling close on the Writer
* (Use the appendMetadata convenience methods). On close, a StoreFile is
* sitting in the Filesystem. To refer to it, create a StoreFile instance
* passing filesystem and path. To read, call {@link #getReader()}.
* <p>StoreFiles may also reference store files in another Store.
*/
public class StoreFile implements HConstants {
static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
// Make default block size for StoreFiles 8k while testing. TODO: FIX!
// Need to make it 8k for testing.
private static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024;
private final FileSystem fs;
// This file's path.
private final Path path;
// If this storefile references another, this is the reference instance.
private Reference reference;
// If this StoreFile references another, this is the other files path.
private Path referencePath;
// Keys for metadata stored in backing HFile.
private static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
// Set when we obtain a Reader.
private long sequenceid = -1;
private static final byte [] MAJOR_COMPACTION_KEY =
Bytes.toBytes("MAJOR_COMPACTION_KEY");
// If true, this file was product of a major compaction. Its then set
// whenever you get a Reader.
private AtomicBoolean majorCompaction = null;
/*
* Regex that will work for straight filenames and for reference names.
* If reference, then the regex has more than just one group. Group 1 is
* this files id. Group 2 the referenced region name, etc.
*/
private static final Pattern REF_NAME_PARSER =
Pattern.compile("^(\\d+)(?:\\.(.+))?$");
private volatile HFile.Reader reader;
// Used making file ids.
private final static Random rand = new Random();
/**
* Constructor.
* Loads up a Reader (and its indices, etc.).
* @param fs Filesystem.
* @param p qualified path
* @throws IOException
*/
StoreFile(final FileSystem fs, final Path p)
throws IOException {
this.fs = fs;
this.path = p;
if (isReference(p)) {
this.reference = Reference.read(fs, p);
this.referencePath = getReferredToFile(this.path);
}
this.reader = open();
}
/**
* @return Path or null if this StoreFile was made with a Stream.
*/
Path getPath() {
return this.path;
}
/**
* @return The Store/ColumnFamily this file belongs to.
*/
byte [] getFamily() {
return Bytes.toBytes(this.path.getParent().getName());
}
/**
* @return True if this is a StoreFile Reference; call after {@link #open()}
* else may get wrong answer.
*/
boolean isReference() {
return this.reference != null;
}
/**
* @param p Path to check.
* @return True if the path has format of a HStoreFile reference.
*/
public static boolean isReference(final Path p) {
return isReference(p, REF_NAME_PARSER.matcher(p.getName()));
}
/**
* @param p Path to check.
* @param m Matcher to use.
* @return True if the path has format of a HStoreFile reference.
*/
public static boolean isReference(final Path p, final Matcher m) {
if (m == null || !m.matches()) {
LOG.warn("Failed match of store file name " + p.toString());
throw new RuntimeException("Failed match of store file name " +
p.toString());
}
return m.groupCount() > 1 && m.group(2) != null;
}
/*
* Return path to the file referred to by a Reference. Presumes a directory
* hierarchy of <code>${hbase.rootdir}/tablename/regionname/familyname</code>.
* @param p Path to a Reference file.
* @return Calculated path to parent region file.
* @throws IOException
*/
static Path getReferredToFile(final Path p) throws IOException {
Matcher m = REF_NAME_PARSER.matcher(p.getName());
if (m == null || !m.matches()) {
LOG.warn("Failed match of store file name " + p.toString());
throw new RuntimeException("Failed match of store file name " +
p.toString());
}
// Other region name is suffix on the passed Reference file name
String otherRegion = m.group(2);
// Tabledir is up two directories from where Reference was written.
Path tableDir = p.getParent().getParent().getParent();
String nameStrippedOfSuffix = m.group(1);
// Build up new path with the referenced region in place of our current
// region in the reference path. Also strip regionname suffix from name.
return new Path(new Path(new Path(tableDir, otherRegion),
p.getParent().getName()), nameStrippedOfSuffix);
}
/**
* @return True if this file was made by a major compaction.
*/
boolean isMajorCompaction() {
if (this.majorCompaction == null) {
throw new NullPointerException("This has not been set yet");
}
return this.majorCompaction.get();
}
/**
* @return This files maximum edit sequence id.
*/
public long getMaxSequenceId() {
if (this.sequenceid == -1) {
throw new IllegalAccessError("Has not been initialized");
}
return this.sequenceid;
}
/**
* Opens reader on this store file. Called by Constructor.
* @return Reader for the store file.
* @throws IOException
* @see #close()
*/
protected HFile.Reader open()
throws IOException {
if (this.reader != null) {
throw new IllegalAccessError("Already open");
}
if (isReference()) {
this.reader = new HalfHFileReader(this.fs, this.referencePath, null,
this.reference);
} else {
this.reader = new HFile.Reader(this.fs, this.path, null);
}
// Load up indices and fileinfo.
Map<byte [], byte []> map = this.reader.loadFileInfo();
// Read in our metadata.
byte [] b = map.get(MAX_SEQ_ID_KEY);
if (b != null) {
// By convention, if halfhfile, top half has a sequence number > bottom
// half. Thats why we add one in below. Its done for case the two halves
// are ever merged back together --rare. Without it, on open of store,
// since store files are distingushed by sequence id, the one half would
// subsume the other.
this.sequenceid = Bytes.toLong(b);
if (isReference()) {
if (Reference.isTopFileRegion(this.reference.getFileRegion())) {
this.sequenceid += 1;
}
}
}
b = map.get(MAJOR_COMPACTION_KEY);
if (b != null) {
boolean mc = Bytes.toBoolean(b);
if (this.majorCompaction == null) {
this.majorCompaction = new AtomicBoolean(mc);
} else {
this.majorCompaction.set(mc);
}
}
return this.reader;
}
/**
* @return Current reader. Must call open first.
*/
public HFile.Reader getReader() {
if (this.reader == null) {
throw new IllegalAccessError("Call open first");
}
return this.reader;
}
/**
* @throws IOException
* @see #open()
*/
public synchronized void close() throws IOException {
if (this.reader != null) {
this.reader.close();
this.reader = null;
}
}
public String toString() {
return this.path.toString() +
(isReference()? "-" + this.referencePath + "-" + reference.toString(): "");
}
/**
* Delete this file
* @throws IOException
*/
public void delete() throws IOException {
close();
this.fs.delete(getPath(), true);
}
/**
* Utility to help with rename.
* @param fs
* @param src
* @param tgt
* @return True if succeeded.
* @throws IOException
*/
public static Path rename(final FileSystem fs, final Path src,
final Path tgt)
throws IOException {
if (!fs.exists(src)) {
throw new FileNotFoundException(src.toString());
}
if (!fs.rename(src, tgt)) {
throw new IOException("Failed rename of " + src + " to " + tgt);
}
return tgt;
}
/**
* Get a store file writer. Client is responsible for closing file when done.
* If metadata, add BEFORE closing using
* {@link #appendMetadata(org.apache.hadoop.hbase.io.hfile.HFile.Writer, long)}.
* @param fs
* @param dir Path to family directory. Makes the directory if doesn't exist.
* Creates a file with a unique name in this directory.
* @return HFile.Writer
* @throws IOException
*/
public static HFile.Writer getWriter(final FileSystem fs, final Path dir)
throws IOException {
return getWriter(fs, dir, DEFAULT_BLOCKSIZE_SMALL, null, null);
}
/**
* Get a store file writer. Client is responsible for closing file when done.
* If metadata, add BEFORE closing using
* {@link #appendMetadata(org.apache.hadoop.hbase.io.hfile.HFile.Writer, long)}.
* @param fs
* @param dir Path to family directory. Makes the directory if doesn't exist.
* Creates a file with a unique name in this directory.
* @param blocksize
* @param algorithm Pass null to get default.
* @param c Pass null to get default.
* @return HFile.Writer
* @throws IOException
*/
public static HFile.Writer getWriter(final FileSystem fs, final Path dir,
final int blocksize, final String algorithm, final RawComparator<byte []> c)
throws IOException {
if (!fs.exists(dir)) {
fs.mkdirs(dir);
}
Path path = getUniqueFile(fs, dir);
return new HFile.Writer(fs, path, blocksize,
algorithm == null? HFile.DEFAULT_COMPRESSION: algorithm,
c == null? HStoreKey.BYTECOMPARATOR: c);
}
/**
* @param fs
* @param p
* @return random filename inside passed <code>dir</code>
*/
static Path getUniqueFile(final FileSystem fs, final Path p)
throws IOException {
if (!fs.getFileStatus(p).isDir()) {
throw new IOException("Expecting a directory");
}
return fs.getFileStatus(p).isDir()? getRandomFilename(fs, p): p;
}
/**
* @param fs
* @param dir
* @param encodedRegionName
* @param family
* @return Path to a file that doesn't exist at time of this invocation.
* @throws IOException
*/
static Path getRandomFilename(final FileSystem fs, final Path dir)
throws IOException {
return getRandomFilename(fs, dir, null);
}
/**
* @param fs
* @param dir
* @param encodedRegionName
* @param family
* @param suffix
* @return Path to a file that doesn't exist at time of this invocation.
* @throws IOException
*/
static Path getRandomFilename(final FileSystem fs, final Path dir,
final String suffix)
throws IOException {
long id = -1;
Path p = null;
do {
id = Math.abs(rand.nextLong());
p = new Path(dir, Long.toString(id) +
((suffix == null || suffix.length() <= 0)? "": suffix));
} while(fs.exists(p));
return p;
}
/**
* Write file metadata.
* Call before you call close on the passed <code>w</code> since its written
* as metadata to that file.
*
* @param filesystem file system
* @param maxSequenceId Maximum sequence id.
* @throws IOException
*/
static void appendMetadata(final HFile.Writer w, final long maxSequenceId)
throws IOException {
appendMetadata(w, maxSequenceId, false);
}
/**
* Writes metadata.
* Call before you call close on the passed <code>w</code> since its written
* as metadata to that file.
* @param maxSequenceId Maximum sequence id.
* @param mc True if this file is product of a major compaction
* @throws IOException
*/
static void appendMetadata(final HFile.Writer w, final long maxSequenceId,
final boolean mc)
throws IOException {
w.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
w.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(mc));
}
/*
* Write out a split reference.
* @param fs
* @param splitDir Presumes path format is actually
* <code>SOME_DIRECTORY/REGIONNAME/FAMILY</code>.
* @param f File to split.
* @param splitRow
* @param range
* @return Path to created reference.
* @throws IOException
*/
static Path split(final FileSystem fs, final Path splitDir,
final StoreFile f, final byte [] splitRow, final Reference.Range range)
throws IOException {
// A reference to the bottom half of the hsf store file.
Reference r = new Reference(new HStoreKey(splitRow).getBytes(), range);
// Add the referred-to regions name as a dot separated suffix.
// See REF_NAME_PARSER regex above. The referred-to regions name is
// up in the path of the passed in <code>f</code> -- parentdir is family,
// then the directory above is the region name.
String parentRegionName = f.getPath().getParent().getParent().getName();
// Write reference with same file id only with the other region name as
// suffix and into the new region location (under same family).
Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
return r.write(fs, p);
}
}

View File

@ -36,13 +36,12 @@ import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.io.MapFile;
/**
* Scanner scans both the memcache and the HStore
*/
class HStoreScanner implements InternalScanner, ChangedReadersObserver {
static final Log LOG = LogFactory.getLog(HStoreScanner.class);
class StoreScanner implements InternalScanner, ChangedReadersObserver {
static final Log LOG = LogFactory.getLog(StoreScanner.class);
private InternalScanner[] scanners;
private TreeMap<byte [], Cell>[] resultSets;
@ -50,7 +49,7 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
private boolean wildcardMatch = false;
private boolean multipleMatchers = false;
private RowFilterInterface dataFilter;
private HStore store;
private Store store;
private final long timestamp;
private final byte [][] targetCols;
@ -66,7 +65,7 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
/** Create an Scanner with a handle on the memcache and HStore files. */
@SuppressWarnings("unchecked")
HStoreScanner(HStore store, byte [][] targetCols, byte [] firstRow,
StoreScanner(Store store, byte [][] targetCols, byte [] firstRow,
long timestamp, RowFilterInterface filter)
throws IOException {
this.store = store;
@ -81,7 +80,6 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
// See updateReaders below.
this.timestamp = timestamp;
this.targetCols = targetCols;
try {
scanners[MEMS_INDEX] =
store.memcache.getScanner(timestamp, targetCols, firstRow);
@ -124,7 +122,8 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
private void setupScanner(final int i) throws IOException {
this.keys[i] = new HStoreKey();
this.resultSets[i] = new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
if (this.scanners[i] != null && !this.scanners[i].next(this.keys[i], this.resultSets[i])) {
if (this.scanners[i] != null && !this.scanners[i].next(this.keys[i],
this.resultSets[i])) {
closeScanner(i);
}
}
@ -154,10 +153,8 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
for (int i = 0; i < this.keys.length; i++) {
if (scanners[i] != null &&
(chosenRow == null ||
(HStoreKey.compareTwoRowKeys(store.getHRegionInfo(),
keys[i].getRow(), chosenRow) < 0) ||
((HStoreKey.compareTwoRowKeys(store.getHRegionInfo(),
keys[i].getRow(), chosenRow) == 0) &&
(HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), chosenRow) < 0) ||
((HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), chosenRow) == 0) &&
(keys[i].getTimestamp() > chosenTimestamp)))) {
chosenRow = keys[i].getRow();
chosenTimestamp = keys[i].getTimestamp();
@ -187,8 +184,7 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
while ((scanners[i] != null
&& !filtered
&& moreToFollow)
&& (HStoreKey.compareTwoRowKeys(store.getHRegionInfo(),
keys[i].getRow(), chosenRow) == 0)) {
&& (HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), chosenRow) == 0)) {
// If we are doing a wild card match or there are multiple
// matchers per column, we need to scan all the older versions of
// this row to pick up the rest of the family members
@ -204,7 +200,7 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
// a result if the map does not contain the key.
HStoreKey hsk = new HStoreKey(key.getRow(),
HConstants.EMPTY_BYTE_ARRAY,
key.getTimestamp(), this.store.getHRegionInfo());
key.getTimestamp());
for (Map.Entry<byte [], Cell> e : resultSets[i].entrySet()) {
hsk.setColumn(e.getKey());
if (HLogEdit.isDeleted(e.getValue().getValue())) {
@ -233,13 +229,11 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
}
}
}
for (int i = 0; i < scanners.length; i++) {
// If the current scanner is non-null AND has a lower-or-equal
// row label, then its timestamp is bad. We need to advance it.
while ((scanners[i] != null) &&
(HStoreKey.compareTwoRowKeys(store.getHRegionInfo(),
keys[i].getRow(), chosenRow) <= 0)) {
(HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), chosenRow) <= 0)) {
resultSets[i].clear();
if (!scanners[i].next(keys[i], resultSets[i])) {
closeScanner(i);
@ -248,7 +242,6 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
}
moreToFollow = chosenTimestamp >= 0;
if (dataFilter != null) {
if (dataFilter.filterAllRemaining()) {
moreToFollow = false;
@ -319,9 +312,8 @@ class HStoreScanner implements InternalScanner, ChangedReadersObserver {
}
this.lock.writeLock().lock();
try {
MapFile.Reader [] readers = this.store.getReaders();
if (this.scanners[HSFS_INDEX] == null && readers != null &&
readers.length > 0) {
Map<Long, StoreFile> map = this.store.getStorefiles();
if (this.scanners[HSFS_INDEX] == null && map != null && map.size() > 0) {
// Presume that we went from no readers to at least one -- need to put
// a HStoreScanner in place.
try {

View File

@ -0,0 +1,365 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.math.random.RandomData;
import org.apache.commons.math.random.RandomDataImpl;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.util.Bytes;
/**
* <p>
* This class runs performance benchmarks for {@link HFile}.
* </p>
*/
public class HFilePerformanceEvaluation {
private static final int ROW_LENGTH = 10;
private static final int ROW_COUNT = 1000000;
private static final int RFILE_BLOCKSIZE = 8 * 1024;
static final Log LOG =
LogFactory.getLog(HFilePerformanceEvaluation.class.getName());
static byte [] format(final int i) {
String v = Integer.toString(i);
return Bytes.toBytes("0000000000".substring(v.length()) + v);
}
static ImmutableBytesWritable format(final int i, ImmutableBytesWritable w) {
w.set(format(i));
return w;
}
private void runBenchmarks() throws Exception {
final Configuration conf = new Configuration();
final FileSystem fs = FileSystem.get(conf);
final Path mf = fs.makeQualified(new Path("performanceevaluation.mapfile"));
if (fs.exists(mf)) {
fs.delete(mf, true);
}
runBenchmark(new SequentialWriteBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
PerformanceEvaluationCommons.concurrentReads(new Runnable() {
public void run() {
try {
runBenchmark(new UniformRandomSmallScan(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
} catch (Exception e) {
e.printStackTrace();
}
}
});
PerformanceEvaluationCommons.concurrentReads(new Runnable() {
public void run() {
try {
runBenchmark(new UniformRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
} catch (Exception e) {
e.printStackTrace();
}
}
});
PerformanceEvaluationCommons.concurrentReads(new Runnable() {
public void run() {
try {
runBenchmark(new GaussianRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
} catch (Exception e) {
e.printStackTrace();
}
}
});
PerformanceEvaluationCommons.concurrentReads(new Runnable() {
public void run() {
try {
runBenchmark(new SequentialReadBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
} catch (Exception e) {
e.printStackTrace();
}
}
});
}
private void runBenchmark(RowOrientedBenchmark benchmark, int rowCount)
throws Exception {
LOG.info("Running " + benchmark.getClass().getSimpleName() + " for " +
rowCount + " rows.");
long elapsedTime = benchmark.run();
LOG.info("Running " + benchmark.getClass().getSimpleName() + " for " +
rowCount + " rows took " + elapsedTime + "ms.");
}
static abstract class RowOrientedBenchmark {
protected final Configuration conf;
protected final FileSystem fs;
protected final Path mf;
protected final int totalRows;
public RowOrientedBenchmark(Configuration conf, FileSystem fs, Path mf,
int totalRows) {
this.conf = conf;
this.fs = fs;
this.mf = mf;
this.totalRows = totalRows;
}
void setUp() throws Exception {
// do nothing
}
abstract void doRow(int i) throws Exception;
protected int getReportingPeriod() {
return this.totalRows / 10;
}
void tearDown() throws Exception {
// do nothing
}
/**
* Run benchmark
* @return elapsed time.
* @throws Exception
*/
long run() throws Exception {
long elapsedTime;
setUp();
long startTime = System.currentTimeMillis();
try {
for (int i = 0; i < totalRows; i++) {
if (i > 0 && i % getReportingPeriod() == 0) {
LOG.info("Processed " + i + " rows.");
}
doRow(i);
}
elapsedTime = System.currentTimeMillis() - startTime;
} finally {
tearDown();
}
return elapsedTime;
}
}
static class SequentialWriteBenchmark extends RowOrientedBenchmark {
protected HFile.Writer writer;
private Random random = new Random();
private byte[] bytes = new byte[ROW_LENGTH];
public SequentialWriteBenchmark(Configuration conf, FileSystem fs, Path mf,
int totalRows) {
super(conf, fs, mf, totalRows);
}
@Override
void setUp() throws Exception {
writer = new HFile.Writer(this.fs, this.mf, RFILE_BLOCKSIZE, null, null);
}
@Override
void doRow(int i) throws Exception {
writer.append(format(i), generateValue());
}
private byte[] generateValue() {
random.nextBytes(bytes);
return bytes;
}
@Override
protected int getReportingPeriod() {
return this.totalRows; // don't report progress
}
@Override
void tearDown() throws Exception {
writer.close();
}
}
static abstract class ReadBenchmark extends RowOrientedBenchmark {
ImmutableBytesWritable key = new ImmutableBytesWritable();
ImmutableBytesWritable value = new ImmutableBytesWritable();
protected HFile.Reader reader;
public ReadBenchmark(Configuration conf, FileSystem fs, Path mf,
int totalRows) {
super(conf, fs, mf, totalRows);
}
@Override
void setUp() throws Exception {
reader = new HFile.Reader(this.fs, this.mf, null);
this.reader.loadFileInfo();
}
@Override
void tearDown() throws Exception {
reader.close();
}
}
static class SequentialReadBenchmark extends ReadBenchmark {
private HFileScanner scanner;
public SequentialReadBenchmark(Configuration conf, FileSystem fs,
Path mf, int totalRows)
throws IOException {
super(conf, fs, mf, totalRows);
}
@Override
void setUp() throws Exception {
super.setUp();
this.scanner = this.reader.getScanner();
this.scanner.seekTo();
}
@Override
void doRow(@SuppressWarnings("unused") int i) throws Exception {
if (this.scanner.next()) {
ByteBuffer k = this.scanner.getKey();
PerformanceEvaluationCommons.assertKey(format(i + 1), k);
ByteBuffer v = scanner.getValue();
PerformanceEvaluationCommons.assertValueSize(v.limit(), ROW_LENGTH);
}
}
@Override
protected int getReportingPeriod() {
return this.totalRows; // don't report progress
}
}
static class UniformRandomReadBenchmark extends ReadBenchmark {
private Random random = new Random();
public UniformRandomReadBenchmark(Configuration conf, FileSystem fs,
Path mf, int totalRows) {
super(conf, fs, mf, totalRows);
}
@Override
void doRow(@SuppressWarnings("unused") int i) throws Exception {
HFileScanner scanner = this.reader.getScanner();
byte [] b = getRandomRow();
scanner.seekTo(b);
ByteBuffer k = scanner.getKey();
PerformanceEvaluationCommons.assertKey(b, k);
ByteBuffer v = scanner.getValue();
PerformanceEvaluationCommons.assertValueSize(v.limit(), ROW_LENGTH);
}
private byte [] getRandomRow() {
return format(random.nextInt(totalRows));
}
}
static class UniformRandomSmallScan extends ReadBenchmark {
private Random random = new Random();
public UniformRandomSmallScan(Configuration conf, FileSystem fs,
Path mf, int totalRows) {
super(conf, fs, mf, totalRows/10);
}
@Override
void doRow(@SuppressWarnings("unused") int i) throws Exception {
HFileScanner scanner = this.reader.getScanner();
byte [] b = getRandomRow();
if (scanner.seekTo(b) != 0) {
System.out.println("Nonexistent row: " + new String(b));
return;
}
ByteBuffer k = scanner.getKey();
PerformanceEvaluationCommons.assertKey(b, k);
// System.out.println("Found row: " + new String(b));
for (int ii = 0; ii < 30; ii++) {
if (!scanner.next()) {
System.out.println("NOTHING FOLLOWS");
}
ByteBuffer v = scanner.getValue();
PerformanceEvaluationCommons.assertValueSize(v.limit(), ROW_LENGTH);
}
}
private byte [] getRandomRow() {
return format(random.nextInt(totalRows));
}
}
static class GaussianRandomReadBenchmark extends ReadBenchmark {
private RandomData randomData = new RandomDataImpl();
public GaussianRandomReadBenchmark(Configuration conf, FileSystem fs,
Path mf, int totalRows) {
super(conf, fs, mf, totalRows);
}
@Override
void doRow(@SuppressWarnings("unused") int i) throws Exception {
HFileScanner scanner = this.reader.getScanner();
scanner.seekTo(getGaussianRandomRowBytes());
for (int ii = 0; ii < 30; ii++) {
if (!scanner.next()) {
System.out.println("NOTHING FOLLOWS");
}
scanner.getKey();
scanner.getValue();
}
}
private byte [] getGaussianRandomRowBytes() {
int r = (int) randomData.nextGaussian(totalRows / 2, totalRows / 10);
return format(r);
}
}
/**
* @param args
* @throws IOException
*/
public static void main(String[] args) throws Exception {
new HFilePerformanceEvaluation().runBenchmarks();
}
}

View File

@ -0,0 +1,78 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
/**
* Code shared by PE tests.
*/
public class PerformanceEvaluationCommons {
static final Log LOG =
LogFactory.getLog(PerformanceEvaluationCommons.class.getName());
public static void assertValueSize(final int expectedSize, final int got) {
if (got != expectedSize) {
throw new AssertionError("Expected " + expectedSize + " but got " + got);
}
}
public static void assertKey(final byte [] expected, final ByteBuffer got) {
byte [] b = new byte[got.limit()];
got.get(b, 0, got.limit());
assertKey(expected, b);
}
public static void assertKey(final byte [] expected, final byte [] got) {
if (!org.apache.hadoop.hbase.util.Bytes.equals(expected, got)) {
throw new AssertionError("Expected " +
org.apache.hadoop.hbase.util.Bytes.toString(expected) +
" but got " + org.apache.hadoop.hbase.util.Bytes.toString(got));
}
}
public static void concurrentReads(final Runnable r) {
final int count = 1;
long now = System.currentTimeMillis();
List<Thread> threads = new ArrayList<Thread>(count);
for (int i = 0; i < count; i++) {
Thread t = new Thread(r);
t.setName("" + i);
threads.add(t);
}
for (Thread t: threads) {
t.start();
}
for (Thread t: threads) {
try {
t.join();
} catch (InterruptedException e) {
e.printStackTrace();
}
}
LOG.info("Test took " + (System.currentTimeMillis() - now));
}
}

View File

@ -0,0 +1,128 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.IOException;
import java.nio.ByteBuffer;
import junit.framework.TestCase;
import org.apache.hadoop.hbase.HStoreKey.StoreKeyByteComparator;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
/**
* Tests for the HStoreKey Plain and Meta RawComparators.
*/
public class TestHStoreKey extends TestCase {
protected void setUp() throws Exception {
super.setUp();
}
protected void tearDown() throws Exception {
super.tearDown();
}
public void testByteBuffer() throws Exception {
final long ts = 123;
final byte [] row = Bytes.toBytes("row");
final byte [] column = Bytes.toBytes("column");
HStoreKey hsk = new HStoreKey(row, column, ts);
ByteBuffer bb = ByteBuffer.wrap(hsk.getBytes());
assertTrue(Bytes.equals(row, HStoreKey.getRow(bb)));
assertTrue(Bytes.equals(column, HStoreKey.getColumn(bb)));
assertEquals(ts, HStoreKey.getTimestamp(bb));
}
/**
* Test the byte comparator works same as the object comparator.
*/
public void testRawComparator() throws IOException {
long timestamp = System.currentTimeMillis();
byte [] a = Bytes.toBytes("a");
HStoreKey past = new HStoreKey(a, a, timestamp - 10);
byte [] pastBytes = Writables.getBytes(past);
HStoreKey now = new HStoreKey(a, a, timestamp);
byte [] nowBytes = Writables.getBytes(now);
HStoreKey future = new HStoreKey(a, a, timestamp + 10);
byte [] futureBytes = Writables.getBytes(future);
StoreKeyByteComparator comparator = new HStoreKey.StoreKeyByteComparator();
assertTrue(past.compareTo(now) > 0);
assertTrue(comparator.compare(pastBytes, nowBytes) > 0);
assertTrue(now.compareTo(now) == 0);
assertTrue(comparator.compare(nowBytes, nowBytes) == 0);
assertTrue(future.compareTo(now) < 0);
assertTrue(comparator.compare(futureBytes, nowBytes) < 0);
// Check that empty column comes before one with a column
HStoreKey nocolumn = new HStoreKey(a, timestamp);
byte [] nocolumnBytes = Writables.getBytes(nocolumn);
HStoreKey withcolumn = new HStoreKey(a, a, timestamp);
byte [] withcolumnBytes = Writables.getBytes(withcolumn);
assertTrue(nocolumn.compareTo(withcolumn) < 0);
assertTrue(comparator.compare(nocolumnBytes, withcolumnBytes) < 0);
// Check that empty column comes and LATEST comes before one with a column
// and old timestamp.
nocolumn = new HStoreKey(a, HConstants.LATEST_TIMESTAMP);
nocolumnBytes = Writables.getBytes(nocolumn);
withcolumn = new HStoreKey(a, a, timestamp);
withcolumnBytes = Writables.getBytes(withcolumn);
assertTrue(nocolumn.compareTo(withcolumn) < 0);
assertTrue(comparator.compare(nocolumnBytes, withcolumnBytes) < 0);
}
// /**
// * Tests cases where rows keys have characters below the ','.
// * See HBASE-832
// * @throws IOException
// */
// public void testHStoreKeyBorderCases() throws IOException {
// HRegionInfo info = new HRegionInfo(new HTableDescriptor("testtable"),
// HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
//
// HStoreKey rowA = new HStoreKey("testtable,www.hbase.org/,1234",
// "", Long.MAX_VALUE, info);
// byte [] rowABytes = Writables.getBytes(rowA);
// HStoreKey rowB = new HStoreKey("testtable,www.hbase.org/%20,99999",
// "", Long.MAX_VALUE, info);
// byte [] rowBBytes = Writables.getBytes(rowB);
// assertTrue(rowA.compareTo(rowB) > 0);
// HStoreKey.Comparator comparator = new HStoreKey.PlainStoreKeyComparator();
// assertTrue(comparator.compare(rowABytes, rowBBytes) > 0);
//
// rowA = new HStoreKey("testtable,www.hbase.org/,1234",
// "", Long.MAX_VALUE, HRegionInfo.FIRST_META_REGIONINFO);
// rowB = new HStoreKey("testtable,www.hbase.org/%20,99999",
// "", Long.MAX_VALUE, HRegionInfo.FIRST_META_REGIONINFO);
// assertTrue(rowA.compareTo(rowB) < 0);
// assertTrue(comparator.compare(rowABytes, rowBBytes) < 0);
//
// rowA = new HStoreKey("testtable,,1234",
// "", Long.MAX_VALUE, HRegionInfo.FIRST_META_REGIONINFO);
// rowB = new HStoreKey("testtable,$www.hbase.org/,99999",
// "", Long.MAX_VALUE, HRegionInfo.FIRST_META_REGIONINFO);
// assertTrue(rowA.compareTo(rowB) < 0);
// assertTrue(comparator.compare(rowABytes, rowBBytes) < 0);
//
// rowA = new HStoreKey(".META.,testtable,www.hbase.org/,1234,4321",
// "", Long.MAX_VALUE, HRegionInfo.ROOT_REGIONINFO);
// rowB = new HStoreKey(".META.,testtable,www.hbase.org/%20,99999,99999",
// "", Long.MAX_VALUE, HRegionInfo.ROOT_REGIONINFO);
// assertTrue(rowA.compareTo(rowB) > 0);
// assertTrue(comparator.compare(rowABytes, rowBBytes) > 0);
// }
}

View File

@ -0,0 +1,111 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.util.Random;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.WritableComparator;
/**
* Generate random <key, value> pairs.
* <p>
* Copied from
* <a href="https://issues.apache.org/jira/browse/HADOOP-3315">hadoop-3315 tfile</a>.
* Remove after tfile is committed and use the tfile version of this class
* instead.</p>
*/
class KVGenerator {
private final Random random;
private final byte[][] dict;
private final boolean sorted;
private final RandomDistribution.DiscreteRNG keyLenRNG, valLenRNG;
private BytesWritable lastKey;
private static final int MIN_KEY_LEN = 4;
private final byte prefix[] = new byte[MIN_KEY_LEN];
public KVGenerator(Random random, boolean sorted,
RandomDistribution.DiscreteRNG keyLenRNG,
RandomDistribution.DiscreteRNG valLenRNG,
RandomDistribution.DiscreteRNG wordLenRNG, int dictSize) {
this.random = random;
dict = new byte[dictSize][];
this.sorted = sorted;
this.keyLenRNG = keyLenRNG;
this.valLenRNG = valLenRNG;
for (int i = 0; i < dictSize; ++i) {
int wordLen = wordLenRNG.nextInt();
dict[i] = new byte[wordLen];
random.nextBytes(dict[i]);
}
lastKey = new BytesWritable();
fillKey(lastKey);
}
private void fillKey(BytesWritable o) {
int len = keyLenRNG.nextInt();
if (len < MIN_KEY_LEN) len = MIN_KEY_LEN;
o.setSize(len);
int n = MIN_KEY_LEN;
while (n < len) {
byte[] word = dict[random.nextInt(dict.length)];
int l = Math.min(word.length, len - n);
System.arraycopy(word, 0, o.get(), n, l);
n += l;
}
if (sorted
&& WritableComparator.compareBytes(lastKey.get(), MIN_KEY_LEN, lastKey
.getSize()
- MIN_KEY_LEN, o.get(), MIN_KEY_LEN, o.getSize() - MIN_KEY_LEN) > 0) {
incrementPrefix();
}
System.arraycopy(prefix, 0, o.get(), 0, MIN_KEY_LEN);
lastKey.set(o);
}
private void fillValue(BytesWritable o) {
int len = valLenRNG.nextInt();
o.setSize(len);
int n = 0;
while (n < len) {
byte[] word = dict[random.nextInt(dict.length)];
int l = Math.min(word.length, len - n);
System.arraycopy(word, 0, o.get(), n, l);
n += l;
}
}
private void incrementPrefix() {
for (int i = MIN_KEY_LEN - 1; i >= 0; --i) {
++prefix[i];
if (prefix[i] != 0) return;
}
throw new RuntimeException("Prefix overflown");
}
public void next(BytesWritable key, BytesWritable value, boolean dupKey) {
if (dupKey) {
key.set(lastKey);
}
else {
fillKey(key);
}
fillValue(value);
}
}

View File

@ -0,0 +1,64 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Random;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.hbase.io.hfile.RandomDistribution.DiscreteRNG;
/*
* <p>
* Copied from
* <a href="https://issues.apache.org/jira/browse/HADOOP-3315">hadoop-3315 tfile</a>.
* Remove after tfile is committed and use the tfile version of this class
* instead.</p>
*/
class KeySampler {
Random random;
int min, max;
DiscreteRNG keyLenRNG;
private static final int MIN_KEY_LEN = 4;
public KeySampler(Random random, byte [] first, byte [] last,
DiscreteRNG keyLenRNG) throws IOException {
this.random = random;
min = keyPrefixToInt(first);
max = keyPrefixToInt(last);
this.keyLenRNG = keyLenRNG;
}
private int keyPrefixToInt(byte [] key) throws IOException {
byte[] b = key;
int o = 0;
return (b[o] & 0xff) << 24 | (b[o + 1] & 0xff) << 16
| (b[o + 2] & 0xff) << 8 | (b[o + 3] & 0xff);
}
public void next(BytesWritable key) {
key.setSize(Math.max(MIN_KEY_LEN, keyLenRNG.nextInt()));
random.nextBytes(key.get());
int n = random.nextInt(max - min) + min;
byte[] b = key.get();
b[0] = (byte) (n >> 24);
b[1] = (byte) (n >> 16);
b[2] = (byte) (n >> 8);
b[3] = (byte) n;
}
}

View File

@ -0,0 +1,198 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hbase.io.hfile;
/**
* A nano-second timer.
* <p>
* Copied from
* <a href="https://issues.apache.org/jira/browse/HADOOP-3315">hadoop-3315 tfile</a>.
* Remove after tfile is committed and use the tfile version of this class
* instead.</p>
*/
public class NanoTimer {
private long last = -1;
private boolean started = false;
private long cumulate = 0;
/**
* Constructor
*
* @param start
* Start the timer upon construction.
*/
public NanoTimer(boolean start) {
if (start) this.start();
}
/**
* Start the timer.
*
* Note: No effect if timer is already started.
*/
public void start() {
if (!this.started) {
this.last = System.nanoTime();
this.started = true;
}
}
/**
* Stop the timer.
*
* Note: No effect if timer is already stopped.
*/
public void stop() {
if (this.started) {
this.started = false;
this.cumulate += System.nanoTime() - this.last;
}
}
/**
* Read the timer.
*
* @return the elapsed time in nano-seconds. Note: If the timer is never
* started before, -1 is returned.
*/
public long read() {
if (!readable()) return -1;
return this.cumulate;
}
/**
* Reset the timer.
*/
public void reset() {
this.last = -1;
this.started = false;
this.cumulate = 0;
}
/**
* Checking whether the timer is started
*
* @return true if timer is started.
*/
public boolean isStarted() {
return this.started;
}
/**
* Format the elapsed time to a human understandable string.
*
* Note: If timer is never started, "ERR" will be returned.
*/
public String toString() {
if (!readable()) {
return "ERR";
}
return NanoTimer.nanoTimeToString(this.cumulate);
}
/**
* A utility method to format a time duration in nano seconds into a human
* understandable stirng.
*
* @param t
* Time duration in nano seconds.
* @return String representation.
*/
public static String nanoTimeToString(long t) {
if (t < 0) return "ERR";
if (t == 0) return "0";
if (t < 1000) {
return t + "ns";
}
double us = (double) t / 1000;
if (us < 1000) {
return String.format("%.2fus", us);
}
double ms = us / 1000;
if (ms < 1000) {
return String.format("%.2fms", ms);
}
double ss = ms / 1000;
if (ss < 1000) {
return String.format("%.2fs", ss);
}
long mm = (long) ss / 60;
ss -= mm * 60;
long hh = mm / 60;
mm -= hh * 60;
long dd = hh / 24;
hh -= dd * 24;
if (dd > 0) {
return String.format("%dd%dh", dd, hh);
}
if (hh > 0) {
return String.format("%dh%dm", hh, mm);
}
if (mm > 0) {
return String.format("%dm%.1fs", mm, ss);
}
return String.format("%.2fs", ss);
/**
* StringBuilder sb = new StringBuilder(); String sep = "";
*
* if (dd > 0) { String unit = (dd > 1) ? "days" : "day";
* sb.append(String.format("%s%d%s", sep, dd, unit)); sep = " "; }
*
* if (hh > 0) { String unit = (hh > 1) ? "hrs" : "hr";
* sb.append(String.format("%s%d%s", sep, hh, unit)); sep = " "; }
*
* if (mm > 0) { String unit = (mm > 1) ? "mins" : "min";
* sb.append(String.format("%s%d%s", sep, mm, unit)); sep = " "; }
*
* if (ss > 0) { String unit = (ss > 1) ? "secs" : "sec";
* sb.append(String.format("%s%.3f%s", sep, ss, unit)); sep = " "; }
*
* return sb.toString();
*/
}
private boolean readable() {
return this.last != -1;
}
/**
* Simple tester.
*
* @param args
*/
public static void main(String[] args) {
long i = 7;
for (int x = 0; x < 20; ++x, i *= 7) {
System.out.println(NanoTimer.nanoTimeToString(i));
}
}
}

View File

@ -0,0 +1,271 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Random;
/**
* A class that generates random numbers that follow some distribution.
* <p>
* Copied from
* <a href="https://issues.apache.org/jira/browse/HADOOP-3315">hadoop-3315 tfile</a>.
* Remove after tfile is committed and use the tfile version of this class
* instead.</p>
*/
public class RandomDistribution {
/**
* Interface for discrete (integer) random distributions.
*/
public static interface DiscreteRNG {
/**
* Get the next random number
*
* @return the next random number.
*/
public int nextInt();
}
/**
* P(i)=1/(max-min)
*/
public static final class Flat implements DiscreteRNG {
private final Random random;
private final int min;
private final int max;
/**
* Generate random integers from min (inclusive) to max (exclusive)
* following even distribution.
*
* @param random
* The basic random number generator.
* @param min
* Minimum integer
* @param max
* maximum integer (exclusive).
*
*/
public Flat(Random random, int min, int max) {
if (min >= max) {
throw new IllegalArgumentException("Invalid range");
}
this.random = random;
this.min = min;
this.max = max;
}
/**
* @see DiscreteRNG#nextInt()
*/
@Override
public int nextInt() {
return random.nextInt(max - min) + min;
}
}
/**
* Zipf distribution. The ratio of the probabilities of integer i and j is
* defined as follows:
*
* P(i)/P(j)=((j-min+1)/(i-min+1))^sigma.
*/
public static final class Zipf implements DiscreteRNG {
private static final double DEFAULT_EPSILON = 0.001;
private final Random random;
private final ArrayList<Integer> k;
private final ArrayList<Double> v;
/**
* Constructor
*
* @param r
* The random number generator.
* @param min
* minimum integer (inclusvie)
* @param max
* maximum integer (exclusive)
* @param sigma
* parameter sigma. (sigma > 1.0)
*/
public Zipf(Random r, int min, int max, double sigma) {
this(r, min, max, sigma, DEFAULT_EPSILON);
}
/**
* Constructor.
*
* @param r
* The random number generator.
* @param min
* minimum integer (inclusvie)
* @param max
* maximum integer (exclusive)
* @param sigma
* parameter sigma. (sigma > 1.0)
* @param epsilon
* Allowable error percentage (0 < epsilon < 1.0).
*/
public Zipf(Random r, int min, int max, double sigma, double epsilon) {
if ((max <= min) || (sigma <= 1) || (epsilon <= 0)
|| (epsilon >= 0.5)) {
throw new IllegalArgumentException("Invalid arguments");
}
random = r;
k = new ArrayList<Integer>();
v = new ArrayList<Double>();
double sum = 0;
int last = -1;
for (int i = min; i < max; ++i) {
sum += Math.exp(-sigma * Math.log(i - min + 1));
if ((last == -1) || i * (1 - epsilon) > last) {
k.add(i);
v.add(sum);
last = i;
}
}
if (last != max - 1) {
k.add(max - 1);
v.add(sum);
}
v.set(v.size() - 1, 1.0);
for (int i = v.size() - 2; i >= 0; --i) {
v.set(i, v.get(i) / sum);
}
}
/**
* @see DiscreteRNG#nextInt()
*/
@Override
public int nextInt() {
double d = random.nextDouble();
int idx = Collections.binarySearch(v, d);
if (idx > 0) {
++idx;
}
else {
idx = -(idx + 1);
}
if (idx >= v.size()) {
idx = v.size() - 1;
}
if (idx == 0) {
return k.get(0);
}
int ceiling = k.get(idx);
int lower = k.get(idx - 1);
return ceiling - random.nextInt(ceiling - lower);
}
}
/**
* Binomial distribution.
*
* P(k)=select(n, k)*p^k*(1-p)^(n-k) (k = 0, 1, ..., n)
*
* P(k)=select(max-min-1, k-min)*p^(k-min)*(1-p)^(k-min)*(1-p)^(max-k-1)
*/
public static final class Binomial implements DiscreteRNG {
private final Random random;
private final int min;
private final int n;
private final double[] v;
private static double select(int n, int k) {
double ret = 1.0;
for (int i = k + 1; i <= n; ++i) {
ret *= (double) i / (i - k);
}
return ret;
}
private static double power(double p, int k) {
return Math.exp(k * Math.log(p));
}
/**
* Generate random integers from min (inclusive) to max (exclusive)
* following Binomial distribution.
*
* @param random
* The basic random number generator.
* @param min
* Minimum integer
* @param max
* maximum integer (exclusive).
* @param p
* parameter.
*
*/
public Binomial(Random random, int min, int max, double p) {
if (min >= max) {
throw new IllegalArgumentException("Invalid range");
}
this.random = random;
this.min = min;
this.n = max - min - 1;
if (n > 0) {
v = new double[n + 1];
double sum = 0.0;
for (int i = 0; i <= n; ++i) {
sum += select(n, i) * power(p, i) * power(1 - p, n - i);
v[i] = sum;
}
for (int i = 0; i <= n; ++i) {
v[i] /= sum;
}
}
else {
v = null;
}
}
/**
* @see DiscreteRNG#nextInt()
*/
@Override
public int nextInt() {
if (v == null) {
return min;
}
double d = random.nextDouble();
int idx = Arrays.binarySearch(v, d);
if (idx > 0) {
++idx;
} else {
idx = -(idx + 1);
}
if (idx >= v.length) {
idx = v.length - 1;
}
return idx + min;
}
}
}

View File

@ -0,0 +1,124 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.BufferedReader;
import java.io.FileReader;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Random seek test.
*/
public class RandomSeek {
private static List<String> slurp(String fname) throws IOException {
BufferedReader istream = new BufferedReader(new FileReader(fname));
String str;
List<String> l = new ArrayList<String>();
while ( (str=istream.readLine()) != null) {
String [] parts = str.split(",");
l.add(parts[0] + ":" + parts[1] + ":" + parts[2]);
}
return l;
}
private static String randKey(List<String> keys) {
Random r = new Random();
//return keys.get(r.nextInt(keys.size()));
return "2" + Integer.toString(7+r.nextInt(2)) + Integer.toString(r.nextInt(100));
//return new String(r.nextInt(100));
}
public static void main(String [] argv) throws IOException {
Configuration conf = new Configuration();
conf.setInt("io.file.buffer.size", 64*1024);
RawLocalFileSystem rlfs = new RawLocalFileSystem();
rlfs.setConf(conf);
LocalFileSystem lfs = new LocalFileSystem(rlfs);
Path path = new Path("/Users/ryan/rfile.big.txt");
long start = System.currentTimeMillis();
SimpleBlockCache cache = new SimpleBlockCache();
//LruBlockCache cache = new LruBlockCache();
Reader reader = new HFile.Reader(lfs, path, cache);
reader.loadFileInfo();
System.out.println(reader.trailer);
long end = System.currentTimeMillis();
System.out.println("Index read time: " + (end - start));
List<String> keys = slurp("/Users/ryan/xaa.50k");
HFileScanner scanner = reader.getScanner();
int count;
long totalBytes = 0;
int notFound = 0;
start = System.nanoTime();
for(count = 0; count < 500000; ++count) {
String key = randKey(keys);
byte [] bkey = Bytes.toBytes(key);
int res = scanner.seekTo(bkey);
if (res == 0) {
ByteBuffer k = scanner.getKey();
ByteBuffer v = scanner.getValue();
totalBytes += k.limit();
totalBytes += v.limit();
} else {
++ notFound;
}
if (res == -1) {
scanner.seekTo();
}
// Scan for another 1000 rows.
for (int i = 0; i < 1000; ++i) {
if (!scanner.next())
break;
ByteBuffer k = scanner.getKey();
ByteBuffer v = scanner.getValue();
totalBytes += k.limit();
totalBytes += v.limit();
}
if ( count % 1000 == 0 ) {
end = System.nanoTime();
System.out.println("Cache block count: " + cache.size() + " dumped: "+ cache.dumps);
//System.out.println("Cache size: " + cache.heapSize());
double msTime = ((end - start) / 1000000.0);
System.out.println("Seeked: "+ count + " in " + msTime + " (ms) "
+ (1000.0 / msTime ) + " seeks/ms "
+ (msTime / 1000.0) + " ms/seek");
start = System.nanoTime();
}
}
System.out.println("Total bytes: " + totalBytes + " not found: " + notFound);
}
}

View File

@ -0,0 +1,248 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.RawComparator;
/**
* test hfile features.
* <p>
* Copied from
* <a href="https://issues.apache.org/jira/browse/HADOOP-3315">hadoop-3315 tfile</a>.
* Remove after tfile is committed and use the tfile version of this class
* instead.</p>
*/
public class TestHFile extends TestCase {
static final Log LOG = LogFactory.getLog(TestHFile.class);
private static String ROOT_DIR =
System.getProperty("test.build.data", "/tmp/TestHFile");
private FileSystem fs;
private Configuration conf;
private final int minBlockSize = 512;
private static String localFormatter = "%010d";
@Override
public void setUp() {
conf = new HBaseConfiguration();
RawLocalFileSystem rawLFS = new RawLocalFileSystem();
rawLFS.setConf(conf);
fs = new LocalFileSystem(rawLFS);
}
// write some records into the tfile
// write them twice
private int writeSomeRecords(Writer writer, int start, int n)
throws IOException {
String value = "value";
for (int i = start; i < (start + n); i++) {
String key = String.format(localFormatter, Integer.valueOf(i));
writer.append(Bytes.toBytes(key), Bytes.toBytes(value + key));
}
return (start + n);
}
private void readAllRecords(HFileScanner scanner) throws IOException {
readAndCheckbytes(scanner, 0, 100);
}
// read the records and check
private int readAndCheckbytes(HFileScanner scanner, int start, int n)
throws IOException {
String value = "value";
int i = start;
for (; i < (start + n); i++) {
ByteBuffer key = scanner.getKey();
ByteBuffer val = scanner.getValue();
String keyStr = String.format(localFormatter, Integer.valueOf(i));
String valStr = value + keyStr;
byte [] keyBytes = Bytes.toBytes(key);
assertTrue("bytes for keys do not match " + keyStr + " " +
Bytes.toString(Bytes.toBytes(key)),
Arrays.equals(Bytes.toBytes(keyStr), keyBytes));
byte [] valBytes = Bytes.toBytes(val);
assertTrue("bytes for vals do not match " + valStr + " " +
Bytes.toString(valBytes),
Arrays.equals(Bytes.toBytes(valStr), valBytes));
if (!scanner.next()) {
break;
}
}
assertEquals(i, start + n - 1);
return (start + n);
}
private byte[] getSomeKey(int rowId) {
return String.format(localFormatter, Integer.valueOf(rowId)).getBytes();
}
private void writeRecords(Writer writer) throws IOException {
writeSomeRecords(writer, 0, 100);
writer.close();
}
private FSDataOutputStream createFSOutput(Path name) throws IOException {
if (fs.exists(name)) fs.delete(name, true);
FSDataOutputStream fout = fs.create(name);
return fout;
}
/**
* test none codecs
*/
void basicWithSomeCodec(String codec) throws IOException {
Path ncTFile = new Path(ROOT_DIR, "basic.hfile");
FSDataOutputStream fout = createFSOutput(ncTFile);
Writer writer = new Writer(fout, minBlockSize, codec, null);
LOG.info(writer);
writeRecords(writer);
fout.close();
FSDataInputStream fin = fs.open(ncTFile);
Reader reader = new Reader(fs.open(ncTFile),
fs.getFileStatus(ncTFile).getLen(), null);
// Load up the index.
reader.loadFileInfo();
LOG.info(reader);
HFileScanner scanner = reader.getScanner();
// Align scanner at start of the file.
scanner.seekTo();
readAllRecords(scanner);
scanner.seekTo(getSomeKey(50));
assertTrue("location lookup failed", scanner.seekTo(getSomeKey(50)) == 0);
// read the key and see if it matches
ByteBuffer readKey = scanner.getKey();
assertTrue("seeked key does not match", Arrays.equals(getSomeKey(50),
Bytes.toBytes(readKey)));
scanner.seekTo(new byte[0]);
ByteBuffer val1 = scanner.getValue();
scanner.seekTo(new byte[0]);
ByteBuffer val2 = scanner.getValue();
assertTrue(Arrays.equals(Bytes.toBytes(val1), Bytes.toBytes(val2)));
reader.close();
fin.close();
fs.delete(ncTFile, true);
}
public void testTFileFeatures() throws IOException {
basicWithSomeCodec("none");
basicWithSomeCodec("gz");
}
private void writeNumMetablocks(Writer writer, int n) {
for (int i = 0; i < n; i++) {
writer.appendMetaBlock("TfileMeta" + i, ("something to test" + i).getBytes());
}
}
private void someTestingWithMetaBlock(Writer writer) {
writeNumMetablocks(writer, 10);
}
private void readNumMetablocks(Reader reader, int n) throws IOException {
for (int i = 0; i < n; i++) {
ByteBuffer b = reader.getMetaBlock("TfileMeta" + i);
byte [] found = Bytes.toBytes(b);
assertTrue("failed to match metadata", Arrays.equals(
("something to test" + i).getBytes(), found));
}
}
private void someReadingWithMetaBlock(Reader reader) throws IOException {
readNumMetablocks(reader, 10);
}
private void metablocks(final String compress) throws Exception {
Path mFile = new Path(ROOT_DIR, "meta.tfile");
FSDataOutputStream fout = createFSOutput(mFile);
Writer writer = new Writer(fout, minBlockSize, compress, null);
someTestingWithMetaBlock(writer);
writer.close();
fout.close();
FSDataInputStream fin = fs.open(mFile);
Reader reader = new Reader(fs.open(mFile), this.fs.getFileStatus(mFile)
.getLen(), null);
reader.loadFileInfo();
// No data -- this should return false.
assertFalse(reader.getScanner().seekTo());
someReadingWithMetaBlock(reader);
fs.delete(mFile, true);
reader.close();
fin.close();
}
// test meta blocks for tfiles
public void testMetaBlocks() throws Exception {
metablocks("none");
metablocks("gz");
}
/**
* Make sure the orginals for our compression libs doesn't change on us.
*/
public void testCompressionOrdinance() {
assertTrue(Compression.Algorithm.LZO.ordinal() == 0);
assertTrue(Compression.Algorithm.GZ.ordinal() == 1);
assertTrue(Compression.Algorithm.NONE.ordinal() == 2);
}
public void testComparator() throws IOException {
Path mFile = new Path(ROOT_DIR, "meta.tfile");
FSDataOutputStream fout = createFSOutput(mFile);
Writer writer = new Writer(fout, minBlockSize, "none",
new RawComparator<byte []>() {
@Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2,
int l2) {
return -Bytes.compareTo(b1, s1, l1, b2, s2, l2);
}
@Override
public int compare(byte[] o1, byte[] o2) {
return compare(o1, 0, o1.length, o2, 0, o2.length);
}
});
writer.append("3".getBytes(), "0".getBytes());
writer.append("2".getBytes(), "0".getBytes());
writer.append("1".getBytes(), "0".getBytes());
writer.close();
}
}

View File

@ -0,0 +1,387 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.Random;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.io.compress.LzoCodec;
/**
* Set of long-running tests to measure performance of HFile.
* <p>
* Copied from
* <a href="https://issues.apache.org/jira/browse/HADOOP-3315">hadoop-3315 tfile</a>.
* Remove after tfile is committed and use the tfile version of this class
* instead.</p>
*/
public class TestHFilePerformance extends TestCase {
private static String ROOT_DIR =
System.getProperty("test.build.data", "/tmp/TestHFilePerformance");
private FileSystem fs;
private Configuration conf;
private long startTimeEpoch;
private long finishTimeEpoch;
private DateFormat formatter;
@Override
public void setUp() throws IOException {
conf = new Configuration();
fs = FileSystem.get(conf);
formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
}
public void startTime() {
startTimeEpoch = System.currentTimeMillis();
System.out.println(formatTime() + " Started timing.");
}
public void stopTime() {
finishTimeEpoch = System.currentTimeMillis();
System.out.println(formatTime() + " Stopped timing.");
}
public long getIntervalMillis() {
return finishTimeEpoch - startTimeEpoch;
}
public void printlnWithTimestamp(String message) {
System.out.println(formatTime() + " " + message);
}
/*
* Format millis into minutes and seconds.
*/
public String formatTime(long milis){
return formatter.format(milis);
}
public String formatTime(){
return formatTime(System.currentTimeMillis());
}
private FSDataOutputStream createFSOutput(Path name) throws IOException {
if (fs.exists(name))
fs.delete(name, true);
FSDataOutputStream fout = fs.create(name);
return fout;
}
//TODO have multiple ways of generating key/value e.g. dictionary words
//TODO to have a sample compressable data, for now, made 1 out of 3 values random
// keys are all random.
private static class KeyValueGenerator {
Random keyRandomizer;
Random valueRandomizer;
long randomValueRatio = 3; // 1 out of randomValueRatio generated values will be random.
long valueSequence = 0 ;
KeyValueGenerator() {
keyRandomizer = new Random(0L); //TODO with seed zero
valueRandomizer = new Random(1L); //TODO with seed one
}
// Key is always random now.
void getKey(byte[] key) {
keyRandomizer.nextBytes(key);
}
void getValue(byte[] value) {
if (valueSequence % randomValueRatio == 0)
valueRandomizer.nextBytes(value);
valueSequence++;
}
}
/**
*
* @param fileType "HFile" or "SequenceFile"
* @param keyLength
* @param valueLength
* @param codecName "none", "lzo", "gz"
* @param rows number of rows to be written.
* @param writeMethod used for HFile only.
* @param minBlockSize used for HFile only.
* @throws IOException
*/
//TODO writeMethod: implement multiple ways of writing e.g. A) known length (no chunk) B) using a buffer and streaming (for many chunks).
public void timeWrite(String fileType, int keyLength, int valueLength,
String codecName, long rows, String writeMethod, int minBlockSize)
throws IOException {
System.out.println("File Type: " + fileType);
System.out.println("Writing " + fileType + " with codecName: " + codecName);
long totalBytesWritten = 0;
//Using separate randomizer for key/value with seeds matching Sequence File.
byte[] key = new byte[keyLength];
byte[] value = new byte[valueLength];
KeyValueGenerator generator = new KeyValueGenerator();
startTime();
Path path = new Path(ROOT_DIR, fileType + ".Performance");
System.out.println(ROOT_DIR + path.getName());
FSDataOutputStream fout = createFSOutput(path);
if ("HFile".equals(fileType)){
System.out.println("HFile write method: ");
HFile.Writer writer =
new HFile.Writer(fout, minBlockSize, codecName, null);
// Writing value in one shot.
for (long l=0 ; l<rows ; l++ ) {
generator.getKey(key);
generator.getValue(value);
writer.append(key, value);
totalBytesWritten += key.length;
totalBytesWritten += value.length;
}
writer.close();
} else if ("SequenceFile".equals(fileType)){
CompressionCodec codec = null;
if ("lzo".equals(codecName))
codec = new LzoCodec();
else if ("gz".equals(codecName))
codec = new GzipCodec();
else if (!"none".equals(codecName))
throw new IOException("Codec not supported.");
SequenceFile.Writer writer;
//TODO
//JobConf conf = new JobConf();
if (!"none".equals(codecName))
writer = SequenceFile.createWriter(conf, fout, BytesWritable.class,
BytesWritable.class, SequenceFile.CompressionType.BLOCK, codec);
else
writer = SequenceFile.createWriter(conf, fout, BytesWritable.class,
BytesWritable.class, SequenceFile.CompressionType.NONE, null);
BytesWritable keyBsw;
BytesWritable valBsw;
for (long l=0 ; l<rows ; l++ ) {
generator.getKey(key);
keyBsw = new BytesWritable(key);
totalBytesWritten += keyBsw.getSize();
generator.getValue(value);
valBsw = new BytesWritable(value);
writer.append(keyBsw, valBsw);
totalBytesWritten += valBsw.getSize();
}
writer.close();
} else
throw new IOException("File Type is not supported");
fout.close();
stopTime();
printlnWithTimestamp("Data written: ");
printlnWithTimestamp(" rate = " +
totalBytesWritten / getIntervalMillis() * 1000 / 1024 / 1024 + "MB/s");
printlnWithTimestamp(" total = " + totalBytesWritten + "B");
printlnWithTimestamp("File written: ");
printlnWithTimestamp(" rate = " +
fs.getFileStatus(path).getLen() / getIntervalMillis() * 1000 / 1024 / 1024 + "MB/s");
printlnWithTimestamp(" total = " + fs.getFileStatus(path).getLen() + "B");
}
public void timeReading(String fileType, int keyLength, int valueLength,
long rows, int method) throws IOException {
System.out.println("Reading file of type: " + fileType);
Path path = new Path(ROOT_DIR, fileType + ".Performance");
System.out.println("Input file size: " + fs.getFileStatus(path).getLen());
long totalBytesRead = 0;
ByteBuffer val;
ByteBuffer key;
startTime();
FSDataInputStream fin = fs.open(path);
if ("HFile".equals(fileType)){
HFile.Reader reader = new HFile.Reader(fs.open(path),
fs.getFileStatus(path).getLen(), null);
reader.loadFileInfo();
System.out.println(reader);
switch (method) {
case 0:
case 1:
default:
{
HFileScanner scanner = reader.getScanner();
scanner.seekTo();
for (long l=0 ; l<rows ; l++ ) {
key = scanner.getKey();
val = scanner.getValue();
totalBytesRead += key.limit() + val.limit();
scanner.next();
}
}
break;
}
} else if("SequenceFile".equals(fileType)){
SequenceFile.Reader reader;
reader = new SequenceFile.Reader(fs, path, new Configuration());
if (reader.getCompressionCodec() != null) {
printlnWithTimestamp("Compression codec class: " + reader.getCompressionCodec().getClass());
} else
printlnWithTimestamp("Compression codec class: " + "none");
BytesWritable keyBsw = new BytesWritable();
BytesWritable valBsw = new BytesWritable();
for (long l=0 ; l<rows ; l++ ) {
reader.next(keyBsw, valBsw);
totalBytesRead += keyBsw.getSize() + valBsw.getSize();
}
reader.close();
//TODO make a tests for other types of SequenceFile reading scenarios
} else {
throw new IOException("File Type not supported.");
}
//printlnWithTimestamp("Closing reader");
fin.close();
stopTime();
//printlnWithTimestamp("Finished close");
printlnWithTimestamp("Finished in " + getIntervalMillis() + "ms");
printlnWithTimestamp("Data read: ");
printlnWithTimestamp(" rate = " +
totalBytesRead / getIntervalMillis() * 1000 / 1024 / 1024 + "MB/s");
printlnWithTimestamp(" total = " + totalBytesRead + "B");
printlnWithTimestamp("File read: ");
printlnWithTimestamp(" rate = " +
fs.getFileStatus(path).getLen() / getIntervalMillis() * 1000 / 1024 / 1024 + "MB/s");
printlnWithTimestamp(" total = " + fs.getFileStatus(path).getLen() + "B");
//TODO uncomment this for final committing so test files is removed.
//fs.delete(path, true);
}
public void testRunComparisons() throws IOException {
int keyLength = 100; // 100B
int valueLength = 5*1024; // 5KB
int minBlockSize = 10*1024*1024; // 10MB
int rows = 10000;
System.out.println("****************************** Sequence File *****************************");
timeWrite("SequenceFile", keyLength, valueLength, "none", rows, null, minBlockSize);
System.out.println("\n+++++++\n");
timeReading("SequenceFile", keyLength, valueLength, rows, -1);
System.out.println("");
System.out.println("----------------------");
System.out.println("");
/* DISABLED LZO
timeWrite("SequenceFile", keyLength, valueLength, "lzo", rows, null, minBlockSize);
System.out.println("\n+++++++\n");
timeReading("SequenceFile", keyLength, valueLength, rows, -1);
System.out.println("");
System.out.println("----------------------");
System.out.println("");
/* Sequence file can only use native hadoop libs gzipping so commenting out.
*/
try {
timeWrite("SequenceFile", keyLength, valueLength, "gz", rows, null,
minBlockSize);
System.out.println("\n+++++++\n");
timeReading("SequenceFile", keyLength, valueLength, rows, -1);
} catch (IllegalArgumentException e) {
System.out.println("Skipping sequencefile gz: " + e.getMessage());
}
System.out.println("\n\n\n");
System.out.println("****************************** HFile *****************************");
timeWrite("HFile", keyLength, valueLength, "none", rows, null, minBlockSize);
System.out.println("\n+++++++\n");
timeReading("HFile", keyLength, valueLength, rows, 0 );
System.out.println("");
System.out.println("----------------------");
System.out.println("");
/* DISABLED LZO
timeWrite("HFile", keyLength, valueLength, "lzo", rows, null, minBlockSize);
System.out.println("\n+++++++\n");
timeReading("HFile", keyLength, valueLength, rows, 0 );
System.out.println("\n+++++++\n");
timeReading("HFile", keyLength, valueLength, rows, 1 );
System.out.println("\n+++++++\n");
timeReading("HFile", keyLength, valueLength, rows, 2 );
System.out.println("");
System.out.println("----------------------");
System.out.println("");
*/
timeWrite("HFile", keyLength, valueLength, "gz", rows, null, minBlockSize);
System.out.println("\n+++++++\n");
timeReading("HFile", keyLength, valueLength, rows, 0 );
System.out.println("\n\n\n\nNotes: ");
System.out.println(" * Timing includes open/closing of files.");
System.out.println(" * Timing includes reading both Key and Value");
System.out.println(" * Data is generated as random bytes. Other methods e.g. using " +
"dictionary with care for distributation of words is under development.");
System.out.println(" * Timing of write currently, includes random value/key generations. " +
"Which is the same for Sequence File and HFile. Another possibility is to generate " +
"test data beforehand");
System.out.println(" * We need to mitigate cache effect on benchmark. We can apply several " +
"ideas, for next step we do a large dummy read between benchmark read to dismantle " +
"caching of data. Renaming of file may be helpful. We can have a loop that reads with" +
" the same method several times and flood cache every time and average it to get a" +
" better number.");
}
}

View File

@ -0,0 +1,510 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Random;
import java.util.StringTokenizer;
import junit.framework.TestCase;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
import org.apache.hadoop.io.BytesWritable;
/**
* test the performance for seek.
* <p>
* Copied from
* <a href="https://issues.apache.org/jira/browse/HADOOP-3315">hadoop-3315 tfile</a>.
* Remove after tfile is committed and use the tfile version of this class
* instead.</p>
*/
public class TestHFileSeek extends TestCase {
private MyOptions options;
private Configuration conf;
private Path path;
private FileSystem fs;
private NanoTimer timer;
private Random rng;
private RandomDistribution.DiscreteRNG keyLenGen;
private KVGenerator kvGen;
@Override
public void setUp() throws IOException {
if (options == null) {
options = new MyOptions(new String[0]);
}
conf = new Configuration();
conf.setInt("tfile.fs.input.buffer.size", options.fsInputBufferSize);
conf.setInt("tfile.fs.output.buffer.size", options.fsOutputBufferSize);
path = new Path(new Path(options.rootDir), options.file);
fs = path.getFileSystem(conf);
timer = new NanoTimer(false);
rng = new Random(options.seed);
keyLenGen =
new RandomDistribution.Zipf(new Random(rng.nextLong()),
options.minKeyLen, options.maxKeyLen, 1.2);
RandomDistribution.DiscreteRNG valLenGen =
new RandomDistribution.Flat(new Random(rng.nextLong()),
options.minValLength, options.maxValLength);
RandomDistribution.DiscreteRNG wordLenGen =
new RandomDistribution.Flat(new Random(rng.nextLong()),
options.minWordLen, options.maxWordLen);
kvGen =
new KVGenerator(rng, true, keyLenGen, valLenGen, wordLenGen,
options.dictSize);
}
@Override
public void tearDown() {
try {
fs.close();
}
catch (Exception e) {
// Nothing
}
}
private static FSDataOutputStream createFSOutput(Path name, FileSystem fs)
throws IOException {
if (fs.exists(name)) {
fs.delete(name, true);
}
FSDataOutputStream fout = fs.create(name);
return fout;
}
private void createTFile() throws IOException {
long totalBytes = 0;
FSDataOutputStream fout = createFSOutput(path, fs);
try {
Writer writer =
new Writer(fout, options.minBlockSize, options.compress, null);
try {
BytesWritable key = new BytesWritable();
BytesWritable val = new BytesWritable();
timer.start();
for (long i = 0; true; ++i) {
if (i % 1000 == 0) { // test the size for every 1000 rows.
if (fs.getFileStatus(path).getLen() >= options.fileSize) {
break;
}
}
kvGen.next(key, val, false);
byte [] k = new byte [key.getLength()];
System.arraycopy(key.getBytes(), 0, k, 0, key.getLength());
byte [] v = new byte [val.getLength()];
System.arraycopy(val.getBytes(), 0, v, 0, key.getLength());
writer.append(k, v);
totalBytes += key.getLength();
totalBytes += val.getLength();
}
timer.stop();
}
finally {
writer.close();
}
}
finally {
fout.close();
}
double duration = (double)timer.read()/1000; // in us.
long fsize = fs.getFileStatus(path).getLen();
System.out.printf(
"time: %s...uncompressed: %.2fMB...raw thrpt: %.2fMB/s\n",
timer.toString(), (double) totalBytes / 1024 / 1024, totalBytes
/ duration);
System.out.printf("time: %s...file size: %.2fMB...disk thrpt: %.2fMB/s\n",
timer.toString(), (double) fsize / 1024 / 1024, fsize / duration);
}
public void seekTFile() throws IOException {
int miss = 0;
long totalBytes = 0;
FSDataInputStream fsdis = fs.open(path);
Reader reader =
new Reader(fsdis, fs.getFileStatus(path).getLen(), null);
reader.loadFileInfo();
System.out.println(reader);
KeySampler kSampler =
new KeySampler(rng, reader.getFirstKey(), reader.getLastKey(),
keyLenGen);
HFileScanner scanner = reader.getScanner();
BytesWritable key = new BytesWritable();
BytesWritable val = new BytesWritable();
timer.reset();
timer.start();
for (int i = 0; i < options.seekCount; ++i) {
kSampler.next(key);
byte [] k = new byte [key.getLength()];
System.arraycopy(key.getBytes(), 0, k, 0, key.getLength());
if (scanner.seekTo(k) >= 0) {
ByteBuffer bbkey = scanner.getKey();
ByteBuffer bbval = scanner.getValue();
totalBytes += bbkey.limit();
totalBytes += bbval.limit();
}
else {
++miss;
}
}
timer.stop();
double duration = (double) timer.read() / 1000; // in us.
System.out.printf(
"time: %s...avg seek: %s...%d hit...%d miss...avg I/O size: %.2fKB\n",
timer.toString(), NanoTimer.nanoTimeToString(timer.read()
/ options.seekCount), options.seekCount - miss, miss,
(double) totalBytes / 1024 / (options.seekCount - miss));
}
public void testSeeks() throws IOException {
if (options.doCreate()) {
createTFile();
}
if (options.doRead()) {
seekTFile();
}
if (options.doCreate()) {
fs.delete(path, true);
}
}
private static class IntegerRange {
private final int from, to;
public IntegerRange(int from, int to) {
this.from = from;
this.to = to;
}
public static IntegerRange parse(String s) throws ParseException {
StringTokenizer st = new StringTokenizer(s, " \t,");
if (st.countTokens() != 2) {
throw new ParseException("Bad integer specification: " + s);
}
int from = Integer.parseInt(st.nextToken());
int to = Integer.parseInt(st.nextToken());
return new IntegerRange(from, to);
}
public int from() {
return from;
}
public int to() {
return to;
}
}
private static class MyOptions {
// hard coded constants
int dictSize = 1000;
int minWordLen = 5;
int maxWordLen = 20;
int osInputBufferSize = 64 * 1024;
int osOutputBufferSize = 64 * 1024;
int fsInputBufferSizeNone = 0;
int fsInputBufferSizeLzo = 0;
int fsInputBufferSizeGz = 0;
int fsOutputBufferSizeNone = 1;
int fsOutputBufferSizeLzo = 1;
int fsOutputBufferSizeGz = 1;
String rootDir =
System.getProperty("test.build.data", "/tmp/TestTFileSeek");
String file = "TestTFileSeek";
// String compress = "lzo"; DISABLED
String compress = "none";
int minKeyLen = 10;
int maxKeyLen = 50;
int minValLength = 1024;
int maxValLength = 2 * 1024;
int minBlockSize = 1 * 1024 * 1024;
int fsOutputBufferSize = 1;
int fsInputBufferSize = 0;
// Default writing 10MB.
long fileSize = 10 * 1024 * 1024;
long seekCount = 1000;
long seed;
static final int OP_CREATE = 1;
static final int OP_READ = 2;
int op = OP_CREATE | OP_READ;
boolean proceed = false;
public MyOptions(String[] args) {
seed = System.nanoTime();
try {
Options opts = buildOptions();
CommandLineParser parser = new GnuParser();
CommandLine line = parser.parse(opts, args, true);
processOptions(line, opts);
validateOptions();
}
catch (ParseException e) {
System.out.println(e.getMessage());
System.out.println("Try \"--help\" option for details.");
setStopProceed();
}
}
public boolean proceed() {
return proceed;
}
private Options buildOptions() {
Option compress =
OptionBuilder.withLongOpt("compress").withArgName("[none|lzo|gz]")
.hasArg().withDescription("compression scheme").create('c');
Option fileSize =
OptionBuilder.withLongOpt("file-size").withArgName("size-in-MB")
.hasArg().withDescription("target size of the file (in MB).")
.create('s');
Option fsInputBufferSz =
OptionBuilder.withLongOpt("fs-input-buffer").withArgName("size")
.hasArg().withDescription(
"size of the file system input buffer (in bytes).").create(
'i');
Option fsOutputBufferSize =
OptionBuilder.withLongOpt("fs-output-buffer").withArgName("size")
.hasArg().withDescription(
"size of the file system output buffer (in bytes).").create(
'o');
Option keyLen =
OptionBuilder
.withLongOpt("key-length")
.withArgName("min,max")
.hasArg()
.withDescription(
"the length range of the key (in bytes)")
.create('k');
Option valueLen =
OptionBuilder
.withLongOpt("value-length")
.withArgName("min,max")
.hasArg()
.withDescription(
"the length range of the value (in bytes)")
.create('v');
Option blockSz =
OptionBuilder.withLongOpt("block").withArgName("size-in-KB").hasArg()
.withDescription("minimum block size (in KB)").create('b');
Option seed =
OptionBuilder.withLongOpt("seed").withArgName("long-int").hasArg()
.withDescription("specify the seed").create('S');
Option operation =
OptionBuilder.withLongOpt("operation").withArgName("r|w|rw").hasArg()
.withDescription(
"action: seek-only, create-only, seek-after-create").create(
'x');
Option rootDir =
OptionBuilder.withLongOpt("root-dir").withArgName("path").hasArg()
.withDescription(
"specify root directory where files will be created.")
.create('r');
Option file =
OptionBuilder.withLongOpt("file").withArgName("name").hasArg()
.withDescription("specify the file name to be created or read.")
.create('f');
Option seekCount =
OptionBuilder
.withLongOpt("seek")
.withArgName("count")
.hasArg()
.withDescription(
"specify how many seek operations we perform (requires -x r or -x rw.")
.create('n');
Option help =
OptionBuilder.withLongOpt("help").hasArg(false).withDescription(
"show this screen").create("h");
return new Options().addOption(compress).addOption(fileSize).addOption(
fsInputBufferSz).addOption(fsOutputBufferSize).addOption(keyLen)
.addOption(blockSz).addOption(rootDir).addOption(valueLen).addOption(
operation).addOption(seekCount).addOption(file).addOption(help);
}
private void processOptions(CommandLine line, Options opts)
throws ParseException {
// --help -h and --version -V must be processed first.
if (line.hasOption('h')) {
HelpFormatter formatter = new HelpFormatter();
System.out.println("TFile and SeqFile benchmark.");
System.out.println();
formatter.printHelp(100,
"java ... TestTFileSeqFileComparison [options]",
"\nSupported options:", opts, "");
return;
}
if (line.hasOption('c')) {
compress = line.getOptionValue('c');
}
if (line.hasOption('d')) {
dictSize = Integer.parseInt(line.getOptionValue('d'));
}
if (line.hasOption('s')) {
fileSize = Long.parseLong(line.getOptionValue('s')) * 1024 * 1024;
}
if (line.hasOption('i')) {
fsInputBufferSize = Integer.parseInt(line.getOptionValue('i'));
}
if (line.hasOption('o')) {
fsOutputBufferSize = Integer.parseInt(line.getOptionValue('o'));
}
if (line.hasOption('n')) {
seekCount = Integer.parseInt(line.getOptionValue('n'));
}
if (line.hasOption('k')) {
IntegerRange ir = IntegerRange.parse(line.getOptionValue('k'));
minKeyLen = ir.from();
maxKeyLen = ir.to();
}
if (line.hasOption('v')) {
IntegerRange ir = IntegerRange.parse(line.getOptionValue('v'));
minValLength = ir.from();
maxValLength = ir.to();
}
if (line.hasOption('b')) {
minBlockSize = Integer.parseInt(line.getOptionValue('b')) * 1024;
}
if (line.hasOption('r')) {
rootDir = line.getOptionValue('r');
}
if (line.hasOption('f')) {
file = line.getOptionValue('f');
}
if (line.hasOption('S')) {
seed = Long.parseLong(line.getOptionValue('S'));
}
if (line.hasOption('x')) {
String strOp = line.getOptionValue('x');
if (strOp.equals("r")) {
op = OP_READ;
}
else if (strOp.equals("w")) {
op = OP_CREATE;
}
else if (strOp.equals("rw")) {
op = OP_CREATE | OP_READ;
}
else {
throw new ParseException("Unknown action specifier: " + strOp);
}
}
proceed = true;
}
private void validateOptions() throws ParseException {
if (!compress.equals("none") && !compress.equals("lzo")
&& !compress.equals("gz")) {
throw new ParseException("Unknown compression scheme: " + compress);
}
if (minKeyLen >= maxKeyLen) {
throw new ParseException(
"Max key length must be greater than min key length.");
}
if (minValLength >= maxValLength) {
throw new ParseException(
"Max value length must be greater than min value length.");
}
if (minWordLen >= maxWordLen) {
throw new ParseException(
"Max word length must be greater than min word length.");
}
return;
}
private void setStopProceed() {
proceed = false;
}
public boolean doCreate() {
return (op & OP_CREATE) != 0;
}
public boolean doRead() {
return (op & OP_READ) != 0;
}
}
public static void main(String[] argv) throws IOException {
TestHFileSeek testCase = new TestHFileSeek();
MyOptions options = new MyOptions(argv);
if (options.proceed == false) {
return;
}
testCase.options = options;
testCase.setUp();
testCase.testSeeks();
testCase.tearDown();
}
}

View File

@ -0,0 +1,146 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import junit.framework.TestCase;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Test {@link HFileScanner#seekTo(byte[])} and its variants.
*/
public class TestSeekTo extends TestCase {
private static String ROOT_DIR =
System.getProperty("test.build.data", "/tmp/TestHFile");
private HBaseConfiguration conf;
private LocalFileSystem fs;
@Override
public void setUp() {
conf = new HBaseConfiguration();
RawLocalFileSystem rawLFS = new RawLocalFileSystem();
rawLFS.setConf(conf);
fs = new LocalFileSystem(rawLFS);
}
private FSDataOutputStream createFSOutput(Path name) throws IOException {
if (fs.exists(name)) fs.delete(name, true);
FSDataOutputStream fout = fs.create(name);
return fout;
}
Path makeNewFile() throws IOException {
Path ncTFile = new Path(ROOT_DIR, "basic.hfile");
FSDataOutputStream fout = createFSOutput(ncTFile);
HFile.Writer writer = new HFile.Writer(fout, 40, "none", null);
// 4 bytes * 3 * 2 for each key/value +
// 3 for keys, 15 for values = 42 (woot)
writer.append(Bytes.toBytes("c"), Bytes.toBytes("value"));
writer.append(Bytes.toBytes("e"), Bytes.toBytes("value"));
writer.append(Bytes.toBytes("g"), Bytes.toBytes("value"));
// block transition
writer.append(Bytes.toBytes("i"), Bytes.toBytes("value"));
writer.append(Bytes.toBytes("k"), Bytes.toBytes("value"));
writer.close();
fout.close();
return ncTFile;
}
public void testSeekBefore() throws Exception {
Path p = makeNewFile();
HFile.Reader reader = new HFile.Reader(fs, p, null);
reader.loadFileInfo();
HFileScanner scanner = reader.getScanner();
assertEquals(false, scanner.seekBefore(Bytes.toBytes("a")));
assertEquals(false, scanner.seekBefore(Bytes.toBytes("c")));
assertEquals(true, scanner.seekBefore(Bytes.toBytes("d")));
assertEquals("c", scanner.getKeyString());
assertEquals(true, scanner.seekBefore(Bytes.toBytes("e")));
assertEquals("c", scanner.getKeyString());
assertEquals(true, scanner.seekBefore(Bytes.toBytes("f")));
assertEquals("e", scanner.getKeyString());
assertEquals(true, scanner.seekBefore(Bytes.toBytes("g")));
assertEquals("e", scanner.getKeyString());
assertEquals(true, scanner.seekBefore(Bytes.toBytes("h")));
assertEquals("g", scanner.getKeyString());
assertEquals(true, scanner.seekBefore(Bytes.toBytes("i")));
assertEquals("g", scanner.getKeyString());
assertEquals(true, scanner.seekBefore(Bytes.toBytes("j")));
assertEquals("i", scanner.getKeyString());
assertEquals(true, scanner.seekBefore(Bytes.toBytes("k")));
assertEquals("i", scanner.getKeyString());
assertEquals(true, scanner.seekBefore(Bytes.toBytes("l")));
assertEquals("k", scanner.getKeyString());
}
public void testSeekTo() throws Exception {
Path p = makeNewFile();
HFile.Reader reader = new HFile.Reader(fs, p, null);
reader.loadFileInfo();
assertEquals(2, reader.blockIndex.count);
HFileScanner scanner = reader.getScanner();
// lies before the start of the file.
assertEquals(-1, scanner.seekTo(Bytes.toBytes("a")));
assertEquals(1, scanner.seekTo(Bytes.toBytes("d")));
assertEquals("c", scanner.getKeyString());
// Across a block boundary now.
assertEquals(1, scanner.seekTo(Bytes.toBytes("h")));
assertEquals("g", scanner.getKeyString());
assertEquals(1, scanner.seekTo(Bytes.toBytes("l")));
assertEquals("k", scanner.getKeyString());
}
public void testBlockContainingKey() throws Exception {
Path p = makeNewFile();
HFile.Reader reader = new HFile.Reader(fs, p, null);
reader.loadFileInfo();
System.out.println(reader.blockIndex.toString());
// falls before the start of the file.
assertEquals(-1, reader.blockIndex.blockContainingKey(Bytes.toBytes("a")));
assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("c")));
assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("d")));
assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("e")));
assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("g")));
assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("h")));
assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("i")));
assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("j")));
assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("k")));
assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("l")));
}
}

View File

@ -1,388 +0,0 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.io.HalfMapFileReader;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.MapFile;
import org.apache.hadoop.hbase.io.SequenceFile;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.io.WritableComparable;
/**
* Test HStoreFile
*/
public class TestHStoreFile extends HBaseTestCase {
static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
private static String DIR = "/";
private MiniDFSCluster cluster;
private Path dir = null;
@Override
public void setUp() throws Exception {
try {
this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
// Set the hbase.rootdir to be the home directory in mini dfs.
this.conf.set(HConstants.HBASE_DIR,
this.cluster.getFileSystem().getHomeDirectory().toString());
this.dir = new Path(DIR, getName());
} catch (IOException e) {
shutdownDfs(cluster);
}
super.setUp();
}
@Override
public void tearDown() throws Exception {
super.tearDown();
shutdownDfs(cluster);
// ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
// "Temporary end-of-test thread dump debugging HADOOP-2040: " + getName());
}
private Path writeMapFile(final String name)
throws IOException {
Path path = new Path(DIR, name);
MapFile.Writer writer = new MapFile.Writer(this.conf, fs, path.toString(),
HStoreKey.class, ImmutableBytesWritable.class);
writeStoreFile(writer);
return path;
}
private Path writeSmallMapFile(final String name)
throws IOException {
Path path = new Path(DIR, name);
MapFile.Writer writer = new MapFile.Writer(this.conf, fs, path.toString(),
HStoreKey.class, ImmutableBytesWritable.class);
try {
for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) {
byte[] b = new byte[] {(byte)d};
byte [] t = Bytes.toBytes(new String(b, HConstants.UTF8_ENCODING));
writer.append(new HStoreKey(t, t, System.currentTimeMillis()),
new ImmutableBytesWritable(t));
}
} finally {
writer.close();
}
return path;
}
/*
* Writes HStoreKey and ImmutableBytes data to passed writer and
* then closes it.
* @param writer
* @throws IOException
*/
private void writeStoreFile(final MapFile.Writer writer)
throws IOException {
try {
for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) {
for (char e = FIRST_CHAR; e <= LAST_CHAR; e++) {
byte[] b = new byte[] { (byte) d, (byte) e };
byte [] t = Bytes.toBytes(new String(b, HConstants.UTF8_ENCODING));
writer.append(new HStoreKey(t, t, System.currentTimeMillis()),
new ImmutableBytesWritable(t));
}
}
} finally {
writer.close();
}
}
/**
* Test that our mechanism of writing store files in one region to reference
* store files in other regions works.
* @throws IOException
*/
public void testReference()
throws IOException {
// Make a store file and write data to it.
HStoreFile hsf = new HStoreFile(this.conf, this.fs, this.dir,
HRegionInfo.FIRST_META_REGIONINFO,
Bytes.toBytes("colfamily"), 1234567890L, null);
MapFile.Writer writer =
hsf.getWriter(this.fs, SequenceFile.CompressionType.NONE, false, 0);
writeStoreFile(writer);
MapFile.Reader reader = hsf.getReader(this.fs, false, false);
// Split on a row, not in middle of row. Midkey returned by reader
// may be in middle of row. Create new one with empty column and
// timestamp.
HStoreKey midkey = new HStoreKey(((HStoreKey)reader.midKey()).getRow());
HStoreKey hsk = new HStoreKey();
reader.finalKey(hsk);
byte [] finalKey = hsk.getRow();
// Make a reference for the bottom half of the just written file.
Reference reference =
new Reference(hsf.getEncodedRegionName(), hsf.getFileId(),
midkey, Reference.Range.top);
HStoreFile refHsf = new HStoreFile(this.conf, this.fs,
new Path(DIR, getName()),
HRegionInfo.FIRST_META_REGIONINFO,
hsf.getColFamily(), 456, reference);
// Assert that reference files are written and that we can write and
// read the info reference file at least.
refHsf.writeReferenceFiles(this.fs);
assertTrue(this.fs.exists(refHsf.getMapFilePath()));
assertTrue(this.fs.exists(refHsf.getInfoFilePath()));
Reference otherReference =
HStoreFile.readSplitInfo(refHsf.getInfoFilePath(), this.fs);
assertEquals(reference.getEncodedRegionName(),
otherReference.getEncodedRegionName());
assertEquals(reference.getFileId(),
otherReference.getFileId());
assertEquals(reference.getMidkey().toString(),
otherReference.getMidkey().toString());
// Now confirm that I can read from the reference and that it only gets
// keys from top half of the file.
MapFile.Reader halfReader = refHsf.getReader(this.fs, false, false);
HStoreKey key = new HStoreKey();
ImmutableBytesWritable value = new ImmutableBytesWritable();
boolean first = true;
while(halfReader.next(key, value)) {
if (first) {
assertTrue(Bytes.equals(key.getRow(), midkey.getRow()));
first = false;
}
}
assertTrue(Bytes.equals(key.getRow(), finalKey));
}
/**
* Write a file and then assert that we can read from top and bottom halves
* using two HalfMapFiles.
* @throws Exception
*/
public void testBasicHalfMapFile() throws Exception {
Path p = writeMapFile(getName());
WritableComparable midkey = getMidkey(p);
checkHalfMapFile(p, midkey);
}
/**
* Check HalfMapFile works even if file we're to go against is smaller than
* the default MapFile interval of 128: i.e. index gets entry every 128
* keys.
* @throws Exception
*/
public void testSmallHalfMapFile() throws Exception {
Path p = writeSmallMapFile(getName());
// I know keys are a-z. Let the midkey we want to use be 'd'. See if
// HalfMapFiles work even if size of file is < than default MapFile
// interval.
checkHalfMapFile(p, new HStoreKey("d"));
}
private WritableComparable getMidkey(final Path p) throws IOException {
MapFile.Reader reader =
new MapFile.Reader(this.fs, p.toString(), this.conf);
HStoreKey key = new HStoreKey();
ImmutableBytesWritable value = new ImmutableBytesWritable();
reader.next(key, value);
String firstKey = key.toString();
WritableComparable midkey = reader.midKey();
reader.finalKey(key);
LOG.info("First key " + firstKey + ", midkey " + midkey.toString()
+ ", last key " + key.toString());
reader.close();
return midkey;
}
private void checkHalfMapFile(final Path p, WritableComparable midkey)
throws IOException {
MapFile.Reader top = null;
MapFile.Reader bottom = null;
HStoreKey key = new HStoreKey();
ImmutableBytesWritable value = new ImmutableBytesWritable();
String previous = null;
try {
// Now make two HalfMapFiles and assert they can read the full backing
// file, one from the top and the other from the bottom.
// Test bottom half first.
bottom = new HalfMapFileReader(this.fs, p.toString(),
this.conf, Reference.Range.bottom, midkey, null);
boolean first = true;
while (bottom.next(key, value)) {
previous = key.toString();
if (first) {
first = false;
LOG.info("First in bottom: " + previous);
}
assertTrue(key.compareTo((HStoreKey)midkey) < 0);
}
if (previous != null) {
LOG.info("Last in bottom: " + previous.toString());
}
// Now test reading from the top.
top = new HalfMapFileReader(this.fs, p.toString(), this.conf,
Reference.Range.top, midkey, null);
first = true;
while (top.next(key, value)) {
assertTrue(key.compareTo((HStoreKey)midkey) >= 0);
if (first) {
first = false;
LOG.info("First in top: " + key.toString());
}
}
LOG.info("Last in top: " + key.toString());
// Next test using a midkey that does not exist in the file.
// First, do a key that is < than first key. Ensure splits behave
// properly.
WritableComparable badkey = new HStoreKey(" ");
bottom = new HalfMapFileReader(this.fs, p.toString(),
this.conf, Reference.Range.bottom, badkey, null);
// When badkey is < than the bottom, should return no values.
assertFalse(bottom.next(key, value));
// Now read from the top.
top = new HalfMapFileReader(this.fs, p.toString(), this.conf,
Reference.Range.top, badkey, null);
first = true;
while (top.next(key, value)) {
assertTrue(key.compareTo((HStoreKey)badkey) >= 0);
if (first) {
first = false;
LOG.info("First top when key < bottom: " + key.toString());
String tmp = Bytes.toString(key.getRow());
for (int i = 0; i < tmp.length(); i++) {
assertTrue(tmp.charAt(i) == 'a');
}
}
}
LOG.info("Last top when key < bottom: " + key.toString());
String tmp = Bytes.toString(key.getRow());
for (int i = 0; i < tmp.length(); i++) {
assertTrue(tmp.charAt(i) == 'z');
}
// Test when badkey is > than last key in file ('||' > 'zz').
badkey = new HStoreKey("|||");
bottom = new HalfMapFileReader(this.fs, p.toString(),
this.conf, Reference.Range.bottom, badkey, null);
first = true;
while (bottom.next(key, value)) {
if (first) {
first = false;
LOG.info("First bottom when key > top: " + key.toString());
tmp = Bytes.toString(key.getRow());
for (int i = 0; i < tmp.length(); i++) {
assertTrue(tmp.charAt(i) == 'a');
}
}
}
LOG.info("Last bottom when key > top: " + key.toString());
tmp = Bytes.toString(key.getRow());
for (int i = 0; i < tmp.length(); i++) {
assertTrue(tmp.charAt(i) == 'z');
}
// Now look at top. Should not return any values.
top = new HalfMapFileReader(this.fs, p.toString(), this.conf,
Reference.Range.top, badkey, null);
assertFalse(top.next(key, value));
} finally {
if (top != null) {
top.close();
}
if (bottom != null) {
bottom.close();
}
fs.delete(p, true);
}
}
/**
* Assert HalFMapFile does right thing when midkey does not exist in the
* backing file (its larger or smaller than any of the backing mapfiles keys).
*
* @throws Exception
*/
public void testOutOfRangeMidkeyHalfMapFile() throws Exception {
MapFile.Reader top = null;
MapFile.Reader bottom = null;
HStoreKey key = new HStoreKey();
ImmutableBytesWritable value = new ImmutableBytesWritable();
Path p = writeMapFile(getName());
try {
try {
// Test using a midkey that does not exist in the file.
// First, do a key that is < than first key. Ensure splits behave
// properly.
HStoreKey midkey = new HStoreKey(" ");
bottom = new HalfMapFileReader(this.fs, p.toString(),
this.conf, Reference.Range.bottom, midkey, null);
// When midkey is < than the bottom, should return no values.
assertFalse(bottom.next(key, value));
// Now read from the top.
top = new HalfMapFileReader(this.fs, p.toString(),
this.conf, Reference.Range.top, midkey, null);
boolean first = true;
while (top.next(key, value)) {
assertTrue(key.compareTo(midkey) >= 0);
if (first) {
first = false;
LOG.info("First top when key < bottom: " + key.toString());
assertEquals("aa", Bytes.toString(key.getRow()));
}
}
LOG.info("Last top when key < bottom: " + key.toString());
assertEquals("zz", Bytes.toString(key.getRow()));
// Test when midkey is > than last key in file ('||' > 'zz').
midkey = new HStoreKey("|||");
bottom = new HalfMapFileReader(this.fs, p.toString(),
this.conf, Reference.Range.bottom, midkey, null);
first = true;
while (bottom.next(key, value)) {
if (first) {
first = false;
LOG.info("First bottom when key > top: " + key.toString());
assertEquals("aa", Bytes.toString(key.getRow()));
}
}
LOG.info("Last bottom when key > top: " + key.toString());
assertEquals("zz", Bytes.toString(key.getRow()));
// Now look at top. Should not return any values.
top = new HalfMapFileReader(this.fs, p.toString(),
this.conf, Reference.Range.top, midkey, null);
assertFalse(top.next(key, value));
} finally {
if (top != null) {
top.close();
}
if (bottom != null) {
bottom.close();
}
fs.delete(p, true);
}
} finally {
this.fs.delete(p, true);
}
}
}

View File

@ -0,0 +1,295 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.io.Reference.Range;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hdfs.MiniDFSCluster;
/**
* Test HStoreFile
*/
public class TestStoreFile extends HBaseTestCase {
static final Log LOG = LogFactory.getLog(TestStoreFile.class);
private MiniDFSCluster cluster;
@Override
public void setUp() throws Exception {
try {
this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
// Set the hbase.rootdir to be the home directory in mini dfs.
this.conf.set(HConstants.HBASE_DIR,
this.cluster.getFileSystem().getHomeDirectory().toString());
} catch (IOException e) {
shutdownDfs(cluster);
}
super.setUp();
}
@Override
public void tearDown() throws Exception {
super.tearDown();
shutdownDfs(cluster);
// ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
// "Temporary end-of-test thread dump debugging HADOOP-2040: " + getName());
}
/**
* Write a file and then assert that we can read from top and bottom halves
* using two HalfMapFiles.
* @throws Exception
*/
public void testBasicHalfMapFile() throws Exception {
// Make up a directory hierarchy that has a regiondir and familyname.
HFile.Writer writer = StoreFile.getWriter(this.fs,
new Path(new Path(this.testDir, "regionname"), "familyname"),
2 * 1024, null, null);
writeStoreFile(writer);
checkHalfHFile(new StoreFile(this.fs, writer.getPath()));
}
/*
* Writes HStoreKey and ImmutableBytes data to passed writer and
* then closes it.
* @param writer
* @throws IOException
*/
private void writeStoreFile(final HFile.Writer writer)
throws IOException {
try {
for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) {
for (char e = FIRST_CHAR; e <= LAST_CHAR; e++) {
byte[] b = new byte[] { (byte) d, (byte) e };
byte [] t = Bytes.toBytes(new String(b, HConstants.UTF8_ENCODING));
HStoreKey hsk = new HStoreKey(t, t, System.currentTimeMillis());
writer.append(hsk.getBytes(), t);
}
}
} finally {
writer.close();
}
}
/**
* Test that our mechanism of writing store files in one region to reference
* store files in other regions works.
* @throws IOException
*/
public void testReference()
throws IOException {
Path storedir = new Path(new Path(this.testDir, "regionname"), "familyname");
Path dir = new Path(storedir, "1234567890");
// Make a store file and write data to it.
HFile.Writer writer = StoreFile.getWriter(this.fs, dir, 8 * 1024, null, null);
writeStoreFile(writer);
StoreFile hsf = new StoreFile(this.fs, writer.getPath());
HFile.Reader reader = hsf.getReader();
// Split on a row, not in middle of row. Midkey returned by reader
// may be in middle of row. Create new one with empty column and
// timestamp.
HStoreKey hsk = HStoreKey.create(reader.midkey());
byte [] midkey = hsk.getRow();
hsk = HStoreKey.create(reader.getLastKey());
byte [] finalKey = hsk.getRow();
// Make a reference
Path refPath = StoreFile.split(fs, dir, hsf, reader.midkey(), Range.top);
StoreFile refHsf = new StoreFile(this.fs, refPath);
// Now confirm that I can read from the reference and that it only gets
// keys from top half of the file.
HFileScanner s = refHsf.getReader().getScanner();
for(boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) {
ByteBuffer bb = s.getKey();
hsk = HStoreKey.create(bb.array(), bb.arrayOffset(), bb.limit());
if (first) {
assertTrue(Bytes.equals(hsk.getRow(), midkey));
first = false;
}
}
assertTrue(Bytes.equals(hsk.getRow(), finalKey));
}
private void checkHalfHFile(final StoreFile f)
throws IOException {
byte [] midkey = f.getReader().midkey();
// Create top split.
Path topDir = Store.getStoreHomedir(this.testDir, 1,
Bytes.toBytes(f.getPath().getParent().getName()));
if (this.fs.exists(topDir)) {
this.fs.delete(topDir, true);
}
Path topPath = StoreFile.split(this.fs, topDir, f, midkey, Range.top);
// Create bottom split.
Path bottomDir = Store.getStoreHomedir(this.testDir, 2,
Bytes.toBytes(f.getPath().getParent().getName()));
if (this.fs.exists(bottomDir)) {
this.fs.delete(bottomDir, true);
}
Path bottomPath = StoreFile.split(this.fs, bottomDir,
f, midkey, Range.bottom);
// Make readers on top and bottom.
HFile.Reader top = new StoreFile(this.fs, topPath).getReader();
HFile.Reader bottom = new StoreFile(this.fs, bottomPath).getReader();
ByteBuffer previous = null;
LOG.info("Midkey: " + Bytes.toString(midkey));
byte [] midkeyBytes = new HStoreKey(midkey).getBytes();
ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midkeyBytes);
try {
// Now make two HalfMapFiles and assert they can read the full backing
// file, one from the top and the other from the bottom.
// Test bottom half first.
// Now test reading from the top.
boolean first = true;
ByteBuffer key = null;
HFileScanner topScanner = top.getScanner();
while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
(topScanner.isSeeked() && topScanner.next())) {
key = topScanner.getKey();
assertTrue(topScanner.getReader().getComparator().compare(key.array(),
key.arrayOffset(), key.limit(), midkeyBytes, 0, midkeyBytes.length) >= 0);
if (first) {
first = false;
LOG.info("First in top: " + Bytes.toString(Bytes.toBytes(key)));
}
}
LOG.info("Last in top: " + Bytes.toString(Bytes.toBytes(key)));
first = true;
HFileScanner bottomScanner = bottom.getScanner();
while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
bottomScanner.next()) {
previous = bottomScanner.getKey();
key = bottomScanner.getKey();
if (first) {
first = false;
LOG.info("First in bottom: " +
Bytes.toString(Bytes.toBytes(previous)));
}
assertTrue(key.compareTo(bbMidkeyBytes) < 0);
}
if (previous != null) {
LOG.info("Last in bottom: " + Bytes.toString(Bytes.toBytes(previous)));
}
// Remove references.
this.fs.delete(topPath, false);
this.fs.delete(bottomPath, false);
// Next test using a midkey that does not exist in the file.
// First, do a key that is < than first key. Ensure splits behave
// properly.
byte [] badmidkey = Bytes.toBytes(" .");
topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
Range.bottom);
top = new StoreFile(this.fs, topPath).getReader();
bottom = new StoreFile(this.fs, bottomPath).getReader();
bottomScanner = bottom.getScanner();
int count = 0;
while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
bottomScanner.next()) {
count++;
}
// When badkey is < than the bottom, should return no values.
assertTrue(count == 0);
// Now read from the top.
first = true;
topScanner = top.getScanner();
while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
topScanner.next()) {
key = topScanner.getKey();
assertTrue(topScanner.getReader().getComparator().compare(key.array(),
key.arrayOffset(), key.limit(), badmidkey, 0, badmidkey.length) >= 0);
if (first) {
first = false;
first = false;
HStoreKey keyhsk = HStoreKey.create(key);
LOG.info("First top when key < bottom: " + keyhsk);
String tmp = Bytes.toString(keyhsk.getRow());
for (int i = 0; i < tmp.length(); i++) {
assertTrue(tmp.charAt(i) == 'a');
}
}
}
HStoreKey keyhsk = HStoreKey.create(key);
LOG.info("Last top when key < bottom: " + keyhsk);
String tmp = Bytes.toString(keyhsk.getRow());
for (int i = 0; i < tmp.length(); i++) {
assertTrue(tmp.charAt(i) == 'z');
}
// Remove references.
this.fs.delete(topPath, false);
this.fs.delete(bottomPath, false);
// Test when badkey is > than last key in file ('||' > 'zz').
badmidkey = Bytes.toBytes("|||");
topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
Range.bottom);
top = new StoreFile(this.fs, topPath).getReader();
bottom = new StoreFile(this.fs, bottomPath).getReader();
first = true;
bottomScanner = bottom.getScanner();
while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
bottomScanner.next()) {
key = bottomScanner.getKey();
if (first) {
first = false;
keyhsk = HStoreKey.create(key);
LOG.info("First bottom when key > top: " + keyhsk);
tmp = Bytes.toString(keyhsk.getRow());
for (int i = 0; i < tmp.length(); i++) {
assertTrue(tmp.charAt(i) == 'a');
}
}
}
keyhsk = HStoreKey.create(key);
LOG.info("Last bottom when key > top: " + keyhsk);
for (int i = 0; i < tmp.length(); i++) {
assertTrue(Bytes.toString(keyhsk.getRow()).charAt(i) == 'z');
}
count = 0;
topScanner = top.getScanner();
while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
(topScanner.isSeeked() && topScanner.next())) {
count++;
}
// When badkey is < than the bottom, should return no values.
assertTrue(count == 0);
} finally {
if (top != null) {
top.close();
}
if (bottom != null) {
bottom.close();
}
fs.delete(f.getPath(), true);
}
}
}