From d29a5b901c6ea80eb495d02100ef8c16bc811a6b Mon Sep 17 00:00:00 2001 From: mbautin Date: Tue, 2 Aug 2011 19:39:22 -0700 Subject: [PATCH] review_hfile-v2-r1153300-git-1152532-2011_08_03_12_47_00 --- .../java/org/apache/hadoop/hbase/HServerLoad.java | 35 +- .../apache/hadoop/hbase/io/DoubleOutputStream.java | 74 + .../hadoop/hbase/io/hfile/AbstractHFileReader.java | 354 ++++ .../hadoop/hbase/io/hfile/AbstractHFileWriter.java | 287 +++ .../apache/hadoop/hbase/io/hfile/BlockCache.java | 26 +- .../apache/hadoop/hbase/io/hfile/BlockType.java | 168 ++ .../apache/hadoop/hbase/io/hfile/CachedBlock.java | 12 +- .../hadoop/hbase/io/hfile/FixedFileTrailer.java | 475 +++++ .../org/apache/hadoop/hbase/io/hfile/HFile.java | 1993 ++------------------ .../apache/hadoop/hbase/io/hfile/HFileBlock.java | 1441 ++++++++++++++ .../hadoop/hbase/io/hfile/HFileBlockIndex.java | 1299 +++++++++++++ .../hadoop/hbase/io/hfile/HFilePrettyPrinter.java | 308 +++ .../hadoop/hbase/io/hfile/HFileReaderV1.java | 666 +++++++ .../hadoop/hbase/io/hfile/HFileReaderV2.java | 732 +++++++ .../hadoop/hbase/io/hfile/HFileWriterV1.java | 483 +++++ .../hadoop/hbase/io/hfile/HFileWriterV2.java | 452 +++++ .../hadoop/hbase/io/hfile/InlineBlockWriter.java | 73 + .../hadoop/hbase/io/hfile/LruBlockCache.java | 32 +- .../hadoop/hbase/io/hfile/SimpleBlockCache.java | 23 +- .../hadoop/hbase/mapreduce/HFileOutputFormat.java | 9 +- .../hbase/mapreduce/LoadIncrementalHFiles.java | 6 +- .../hadoop/hbase/regionserver/HRegionServer.java | 35 +- .../apache/hadoop/hbase/regionserver/Store.java | 39 +- .../hadoop/hbase/regionserver/StoreFile.java | 365 +++-- .../regionserver/metrics/RegionServerMetrics.java | 26 +- .../org/apache/hadoop/hbase/util/BloomFilter.java | 96 +- .../apache/hadoop/hbase/util/BloomFilterBase.java | 56 + .../hadoop/hbase/util/BloomFilterFactory.java | 208 ++ .../hadoop/hbase/util/BloomFilterWriter.java | 61 + .../apache/hadoop/hbase/util/ByteBloomFilter.java | 460 ++++- .../java/org/apache/hadoop/hbase/util/Bytes.java | 54 +- .../hadoop/hbase/util/CompoundBloomFilter.java | 175 ++ .../hadoop/hbase/util/CompoundBloomFilterBase.java | 95 + .../hbase/util/CompoundBloomFilterWriter.java | 277 +++ .../apache/hadoop/hbase/util/CompressionTest.java | 7 +- .../hadoop/hbase/util/DynamicByteBloomFilter.java | 302 --- .../java/org/apache/hadoop/hbase/util/Hash.java | 4 +- .../java/org/apache/hadoop/hbase/util/IdLock.java | 120 ++ .../hbase/zookeeper/RecoverableZooKeeper.java | 4 +- src/main/resources/hbase-default.xml | 51 + .../apache/hadoop/hbase/HBaseTestingUtility.java | 26 + .../hadoop/hbase/HFilePerformanceEvaluation.java | 7 +- .../hadoop/hbase/io/TestHalfStoreFileReader.java | 9 +- .../apache/hadoop/hbase/io/hfile/RandomSeek.java | 4 +- .../hadoop/hbase/io/hfile/TestCacheOnWrite.java | 229 +++ .../hbase/io/hfile/TestCachedBlockQueue.java | 11 +- .../hbase/io/hfile/TestFixedFileTrailer.java | 227 +++ .../apache/hadoop/hbase/io/hfile/TestHFile.java | 72 +- .../hadoop/hbase/io/hfile/TestHFileBlock.java | 499 +++++ .../hadoop/hbase/io/hfile/TestHFileBlockIndex.java | 602 ++++++ .../hbase/io/hfile/TestHFilePerformance.java | 6 +- .../hadoop/hbase/io/hfile/TestHFileReaderV1.java | 89 + .../hadoop/hbase/io/hfile/TestHFileSeek.java | 8 +- .../hadoop/hbase/io/hfile/TestHFileWriterV2.java | 256 +++ .../hadoop/hbase/io/hfile/TestLruBlockCache.java | 160 +- .../apache/hadoop/hbase/io/hfile/TestReseekTo.java | 5 +- .../apache/hadoop/hbase/io/hfile/TestSeekTo.java | 58 +- .../hbase/mapreduce/TestHFileOutputFormat.java | 8 +- .../hbase/mapreduce/TestLoadIncrementalHFiles.java | 12 +- .../regionserver/TestCompoundBloomFilter.java | 353 ++++ .../hbase/regionserver/TestFSErrorsExposed.java | 6 +- .../hadoop/hbase/regionserver/TestStore.java | 12 +- .../hadoop/hbase/regionserver/TestStoreFile.java | 77 +- .../hbase/regionserver/wal/TestWALReplay.java | 2 +- .../hadoop/hbase/util/TestByteBloomFilter.java | 59 +- .../org/apache/hadoop/hbase/util/TestBytes.java | 48 +- .../org/apache/hadoop/hbase/util/TestIdLock.java | 111 ++ .../io/hfile/8e8ab58dcf39412da19833fcd8f687ac | Bin 0 -> 84964 bytes 68 files changed, 11630 insertions(+), 2709 deletions(-) create mode 100644 src/main/java/org/apache/hadoop/hbase/io/DoubleOutputStream.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java create mode 100644 src/main/java/org/apache/hadoop/hbase/io/hfile/InlineBlockWriter.java create mode 100644 src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java create mode 100644 src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java create mode 100644 src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java create mode 100644 src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java create mode 100644 src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java create mode 100644 src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java delete mode 100644 src/main/java/org/apache/hadoop/hbase/util/DynamicByteBloomFilter.java create mode 100644 src/main/java/org/apache/hadoop/hbase/util/IdLock.java create mode 100644 src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java create mode 100644 src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java create mode 100644 src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java create mode 100644 src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java create mode 100644 src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderV1.java create mode 100644 src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java create mode 100644 src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java create mode 100644 src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java create mode 100644 src/test/resources/org/apache/hadoop/hbase/io/hfile/8e8ab58dcf39412da19833fcd8f687ac diff --git a/src/main/java/org/apache/hadoop/hbase/HServerLoad.java b/src/main/java/org/apache/hadoop/hbase/HServerLoad.java index 6c1d33d..78581b9 100644 --- a/src/main/java/org/apache/hadoop/hbase/HServerLoad.java +++ b/src/main/java/org/apache/hadoop/hbase/HServerLoad.java @@ -83,13 +83,29 @@ implements WritableComparable { private int storefileSizeMB; /** the current size of the memstore for the region, in MB */ private int memstoreSizeMB; - /** the current total size of storefile indexes for the region, in MB */ + + /** + * The current total size of root-level store file indexes for the region, + * in MB. The same as {@link #rootIndexSizeKB} but in MB. + */ private int storefileIndexSizeMB; /** the current total read requests made to region */ private int readRequestsCount; /** the current total write requests made to region */ private int writeRequestsCount; + /** The current total size of root-level indexes for the region, in KB. */ + private int rootIndexSizeKB; + + /** The total size of all index blocks, not just the root level, in KB. */ + private int totalStaticIndexSizeKB; + + /** + * The total size of all Bloom filter blocks, not just loaded into the + * block cache, in KB. + */ + private int totalStaticBloomSizeKB; + /** * Constructor, for Writable */ @@ -111,6 +127,8 @@ implements WritableComparable { final int storefiles, final int storeUncompressedSizeMB, final int storefileSizeMB, final int memstoreSizeMB, final int storefileIndexSizeMB, + final int rootIndexSizeKB, final int totalStaticIndexSizeKB, + final int totalStaticBloomSizeKB, final int readRequestsCount, final int writeRequestsCount) { this.name = name; this.stores = stores; @@ -119,6 +137,9 @@ implements WritableComparable { this.storefileSizeMB = storefileSizeMB; this.memstoreSizeMB = memstoreSizeMB; this.storefileIndexSizeMB = storefileIndexSizeMB; + this.rootIndexSizeKB = rootIndexSizeKB; + this.totalStaticIndexSizeKB = totalStaticIndexSizeKB; + this.totalStaticBloomSizeKB = totalStaticBloomSizeKB; this.readRequestsCount = readRequestsCount; this.writeRequestsCount = writeRequestsCount; } @@ -263,6 +284,9 @@ implements WritableComparable { this.storefileIndexSizeMB = in.readInt(); this.readRequestsCount = in.readInt(); this.writeRequestsCount = in.readInt(); + this.rootIndexSizeKB = in.readInt(); + this.totalStaticIndexSizeKB = in.readInt(); + this.totalStaticBloomSizeKB = in.readInt(); } public void write(DataOutput out) throws IOException { @@ -278,6 +302,9 @@ implements WritableComparable { out.writeInt(storefileIndexSizeMB); out.writeInt(readRequestsCount); out.writeInt(writeRequestsCount); + out.writeInt(rootIndexSizeKB); + out.writeInt(totalStaticIndexSizeKB); + out.writeInt(totalStaticBloomSizeKB); } /** @@ -306,6 +333,12 @@ implements WritableComparable { Long.valueOf(this.readRequestsCount)); sb = Strings.appendKeyValue(sb, "writeRequestsCount", Long.valueOf(this.writeRequestsCount)); + sb = Strings.appendKeyValue(sb, "rootIndexSizeKB", + Integer.valueOf(this.rootIndexSizeKB)); + sb = Strings.appendKeyValue(sb, "totalStaticIndexSizeKB", + Integer.valueOf(this.totalStaticIndexSizeKB)); + sb = Strings.appendKeyValue(sb, "totalStaticBloomSizeKB", + Integer.valueOf(this.totalStaticBloomSizeKB)); return sb.toString(); } } diff --git a/src/main/java/org/apache/hadoop/hbase/io/DoubleOutputStream.java b/src/main/java/org/apache/hadoop/hbase/io/DoubleOutputStream.java new file mode 100644 index 0000000..cf33cd3 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/DoubleOutputStream.java @@ -0,0 +1,74 @@ +/* + * Copyright 2011 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.io.OutputStream; + +/** + * An output stream that writes to two streams on each operation. Does not + * attempt to handle exceptions gracefully. If any operation other than + * {@link #close()} fails on the first stream, it is not called on the second + * stream. + */ +public class DoubleOutputStream extends OutputStream { + private OutputStream out1; + private OutputStream out2; + + public DoubleOutputStream(OutputStream out1, OutputStream out2) { + this.out1 = out1; + this.out2 = out2; + } + + @Override + public void write(int b) throws IOException { + out1.write(b); + out2.write(b); + } + + @Override + public void write(byte b[]) throws IOException { + out1.write(b, 0, b.length); + out2.write(b, 0, b.length); + } + + @Override + public void write(byte b[], int off, int len) throws IOException { + out1.write(b, off, len); + out2.write(b, off, len); + } + + @Override + public void flush() throws IOException { + out1.flush(); + out2.flush(); + } + + @Override + public void close() throws IOException { + try { + out1.close(); + } finally { + // Make sure we at least attempt to close both streams. + out2.close(); + } + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java new file mode 100644 index 0000000..f5173c4 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java @@ -0,0 +1,354 @@ +/* + * Copyright 2011 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.concurrent.atomic.AtomicLong; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; +import org.apache.hadoop.hbase.io.hfile.HFile.Reader; +import org.apache.hadoop.io.RawComparator; + +/** + * Common functionality needed by all versions of {@link HFile} readers. + */ +public abstract class AbstractHFileReader implements HFile.Reader { + + private static final Log LOG = LogFactory.getLog(AbstractHFileReader.class); + + /** Filesystem-level block reader for this HFile format version. */ + protected HFileBlock.FSReader fsBlockReader; + + /** Stream to read from. */ + protected FSDataInputStream istream; + + /** + * True if we should close the input stream when done. We don't close it if we + * didn't open it. + */ + protected final boolean closeIStream; + + /** Data block index reader keeping the root data index in memory */ + protected HFileBlockIndex.BlockIndexReader dataBlockIndexReader; + + /** Meta block index reader -- always single level */ + protected HFileBlockIndex.BlockIndexReader metaBlockIndexReader; + + protected final FixedFileTrailer trailer; + + /** Filled when we read in the trailer. */ + protected final Compression.Algorithm compressAlgo; + + /** Last key in the file. Filled in when we read in the file info */ + protected byte [] lastKey = null; + + /** Average key length read from file info */ + protected int avgKeyLen = -1; + + /** Average value length read from file info */ + protected int avgValueLen = -1; + + /** Key comparator */ + protected RawComparator comparator; + + /** Size of this file. */ + protected final long fileSize; + + /** Block cache to use. */ + protected final BlockCache blockCache; + + protected AtomicLong cacheHits = new AtomicLong(); + protected AtomicLong blockLoads = new AtomicLong(); + protected AtomicLong metaLoads = new AtomicLong(); + + /** + * Whether file is from in-memory store (comes from column family + * configuration). + */ + protected boolean inMemory = false; + + /** + * Whether blocks of file should be evicted from the block cache when the + * file is being closed + */ + protected final boolean evictOnClose; + + /** Path of file */ + protected final Path path; + + /** File name to be used for block names */ + protected final String name; + + protected FileInfo fileInfo; + + /** Prefix of the form cf. for statistics counters. */ + private final String cfStatsPrefix; + + protected AbstractHFileReader(Path path, FixedFileTrailer trailer, + final FSDataInputStream fsdis, final long fileSize, + final boolean closeIStream, + final BlockCache blockCache, final boolean inMemory, + final boolean evictOnClose) { + this.trailer = trailer; + this.compressAlgo = trailer.getCompressionCodec(); + this.blockCache = blockCache; + this.fileSize = fileSize; + this.istream = fsdis; + this.closeIStream = closeIStream; + this.inMemory = inMemory; + this.evictOnClose = evictOnClose; + this.path = path; + this.name = path.getName(); + cfStatsPrefix = "cf." + parseCfNameFromPath(path.toString()); + } + + @SuppressWarnings("serial") + public static class BlockIndexNotLoadedException + extends IllegalStateException { + public BlockIndexNotLoadedException() { + // Add a message in case anyone relies on it as opposed to class name. + super("Block index not loaded"); + } + } + + protected String toStringFirstKey() { + return KeyValue.keyToString(getFirstKey()); + } + + protected String toStringLastKey() { + return KeyValue.keyToString(getLastKey()); + } + + /** + * Parse the HFile path to figure out which table and column family + * it belongs to. This is used to maintain read statistics on a + * per-column-family basis. + * + * @param path HFile path name + */ + public static String parseCfNameFromPath(String path) { + String splits[] = path.split("/"); + if (splits.length < 2) { + LOG.warn("Could not determine the table and column family of the " + + "HFile path " + path); + return "unknown"; + } + + return splits[splits.length - 2]; + } + + public abstract boolean isFileInfoLoaded(); + + @Override + public String toString() { + return "reader=" + path.toString() + + (!isFileInfoLoaded()? "": + ", compression=" + compressAlgo.getName() + + ", inMemory=" + inMemory + + ", firstKey=" + toStringFirstKey() + + ", lastKey=" + toStringLastKey()) + + ", avgKeyLen=" + avgKeyLen + + ", avgValueLen=" + avgValueLen + + ", entries=" + trailer.getEntryCount() + + ", length=" + fileSize; + } + + @Override + public long length() { + return fileSize; + } + + /** + * Create a Scanner on this file. No seeks or reads are done on creation. Call + * {@link HFileScanner#seekTo(byte[])} to position an start the read. There is + * nothing to clean up in a Scanner. Letting go of your references to the + * scanner is sufficient. NOTE: Do not use this overload of getScanner for + * compactions. + * + * @param cacheBlocks True if we should cache blocks read in by this scanner. + * @param pread Use positional read rather than seek+read if true (pread is + * better for random reads, seek+read is better scanning). + * @return Scanner on this file. + */ + @Override + public HFileScanner getScanner(boolean cacheBlocks, final boolean pread) { + return getScanner(cacheBlocks, pread, false); + } + + /** + * @return the first key in the file. May be null if file has no entries. Note + * that this is not the first row key, but rather the byte form of the + * first KeyValue. + */ + @Override + public byte [] getFirstKey() { + if (dataBlockIndexReader == null) { + throw new BlockIndexNotLoadedException(); + } + return dataBlockIndexReader.isEmpty() ? null + : dataBlockIndexReader.getRootBlockKey(0); + } + + /** + * TODO left from {@HFile} version 1: move this to StoreFile after Ryan's + * patch goes in to eliminate {@link KeyValue} here. + * + * @return the first row key, or null if the file is empty. + */ + @Override + public byte[] getFirstRowKey() { + byte[] firstKey = getFirstKey(); + if (firstKey == null) + return null; + return KeyValue.createKeyValueFromKey(firstKey).getRow(); + } + + /** + * TODO left from {@HFile} version 1: move this to StoreFile after + * Ryan's patch goes in to eliminate {@link KeyValue} here. + * + * @return the last row key, or null if the file is empty. + */ + @Override + public byte[] getLastRowKey() { + byte[] lastKey = getLastKey(); + if (lastKey == null) + return null; + return KeyValue.createKeyValueFromKey(lastKey).getRow(); + } + + /** @return number of KV entries in this HFile */ + @Override + public long getEntries() { + return trailer.getEntryCount(); + } + + /** @return comparator */ + @Override + public RawComparator getComparator() { + return comparator; + } + + /** @return compression algorithm */ + @Override + public Compression.Algorithm getCompressionAlgorithm() { + return compressAlgo; + } + + /** + * @return the total heap size of data and meta block indexes in bytes. Does + * not take into account non-root blocks of a multilevel data index. + */ + public long indexSize() { + return (dataBlockIndexReader != null ? dataBlockIndexReader.heapSize() : 0) + + ((metaBlockIndexReader != null) ? metaBlockIndexReader.heapSize() + : 0); + } + + @Override + public String getName() { + return name; + } + + @Override + public HFileBlockIndex.BlockIndexReader getDataBlockIndexReader() { + return dataBlockIndexReader; + } + + @Override + public String getColumnFamilyName() { + return cfStatsPrefix; + } + + @Override + public FixedFileTrailer getTrailer() { + return trailer; + } + + @Override + public FileInfo loadFileInfo() throws IOException { + return fileInfo; + } + + /** + * An exception thrown when an operation requiring a scanner to be seeked + * is invoked on a scanner that is not seeked. + */ + @SuppressWarnings("serial") + public static class NotSeekedException extends IllegalStateException { + public NotSeekedException() { + super("Not seeked to a key/value"); + } + } + + protected static abstract class Scanner implements HFileScanner { + protected HFile.Reader reader; + protected ByteBuffer blockBuffer; + + protected boolean cacheBlocks; + protected final boolean pread; + protected final boolean isCompaction; + + protected int currKeyLen; + protected int currValueLen; + + protected int blockFetches; + + public Scanner(final HFile.Reader reader, final boolean cacheBlocks, + final boolean pread, final boolean isCompaction) { + this.reader = reader; + this.cacheBlocks = cacheBlocks; + this.pread = pread; + this.isCompaction = isCompaction; + } + + @Override + public Reader getReader() { + return reader; + } + + @Override + public boolean isSeeked(){ + return blockBuffer != null; + } + + @Override + public String toString() { + return "HFileScanner for reader " + String.valueOf(reader); + } + + protected void assertSeeked() { + if (!isSeeked()) + throw new NotSeekedException(); + } + } + + /** For testing */ + HFileBlock.FSReader getUncachedBlockReader() { + return fsBlockReader; + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java new file mode 100644 index 0000000..2b3329d --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java @@ -0,0 +1,287 @@ +/* + * Copyright 2011 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.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.KeyValue.KeyComparator; +import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.RawComparator; +import org.apache.hadoop.io.Writable; + +/** + * Common functionality needed by all versions of {@link HFile} writers. + */ +public abstract class AbstractHFileWriter implements HFile.Writer { + + /** Key previously appended. Becomes the last key in the file. */ + protected byte[] lastKeyBuffer = null; + + protected int lastKeyOffset = -1; + protected int lastKeyLength = -1; + + /** FileSystem stream to write into. */ + protected FSDataOutputStream outputStream; + + /** True if we opened the outputStream (and so will close it). */ + protected final boolean closeOutputStream; + + /** A "file info" block: a key-value map of file-wide metadata. */ + protected FileInfo fileInfo = new HFile.FileInfo(); + + /** Number of uncompressed bytes we allow per block. */ + protected final int blockSize; + + /** Total # of key/value entries, i.e. how many times add() was called. */ + protected long entryCount = 0; + + /** Used for calculating the average key length. */ + protected long totalKeyLength = 0; + + /** Used for calculating the average value length. */ + protected long totalValueLength = 0; + + /** Total uncompressed bytes, maybe calculate a compression ratio later. */ + protected long totalUncompressedBytes = 0; + + /** Key comparator. Used to ensure we write in order. */ + protected final RawComparator comparator; + + /** Meta block names. */ + protected List metaNames = new ArrayList(); + + /** {@link Writable}s representing meta block data. */ + protected List metaData = new ArrayList(); + + /** The compression algorithm used. NONE if no compression. */ + protected final Compression.Algorithm compressAlgo; + + /** First key in a block. */ + protected byte[] firstKeyInBlock = null; + + /** May be null if we were passed a stream. */ + protected final Path path; + + /** Whether to cache key/value data blocks on write */ + protected final boolean cacheDataBlocksOnWrite; + + /** Whether to cache non-root index blocks on write */ + protected final boolean cacheIndexBlocksOnWrite; + + /** Block cache to optionally fill on write. */ + protected BlockCache blockCache; + + /** Configuration used for block cache initialization */ + private Configuration conf; + + /** + * Name for this object used when logging or in toString. Is either + * the result of a toString on stream or else toString of passed file Path. + */ + protected final String name; + + public AbstractHFileWriter(Configuration conf, + FSDataOutputStream outputStream, Path path, int blockSize, + Compression.Algorithm compressAlgo, KeyComparator comparator) { + this.outputStream = outputStream; + this.path = path; + this.name = path != null ? path.getName() : outputStream.toString(); + this.blockSize = blockSize; + this.compressAlgo = compressAlgo == null + ? HFile.DEFAULT_COMPRESSION_ALGORITHM : compressAlgo; + this.comparator = comparator != null ? comparator + : Bytes.BYTES_RAWCOMPARATOR; + + closeOutputStream = path != null; + + cacheDataBlocksOnWrite = conf.getBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, + false); + cacheIndexBlocksOnWrite = HFileBlockIndex.shouldCacheOnWrite(conf); + + this.conf = conf; + + if (cacheDataBlocksOnWrite || cacheIndexBlocksOnWrite) + initBlockCache(); + } + + /** + * Add last bits of metadata to file info before it is written out. + */ + protected void finishFileInfo() throws IOException { + if (lastKeyBuffer != null) { + // Make a copy. The copy is stuffed into HMapWritable. Needs a clean + // byte buffer. Won't take a tuple. + fileInfo.append(FileInfo.LASTKEY, Arrays.copyOfRange(lastKeyBuffer, + lastKeyOffset, lastKeyOffset + lastKeyLength), false); + } + + // Average key length. + int avgKeyLen = + entryCount == 0 ? 0 : (int) (totalKeyLength / entryCount); + fileInfo.append(FileInfo.AVG_KEY_LEN, Bytes.toBytes(avgKeyLen), false); + + // Average value length. + int avgValueLen = + entryCount == 0 ? 0 : (int) (totalValueLength / entryCount); + fileInfo.append(FileInfo.AVG_VALUE_LEN, Bytes.toBytes(avgValueLen), false); + } + + /** + * Add to the file info. All added key/value pairs can be obtained using + * {@link HFile.Reader#loadFileInfo()}. + * + * @param k Key + * @param v Value + * @throws IOException in case the key or the value are invalid + */ + @Override + public void appendFileInfo(final byte[] k, final byte[] v) + throws IOException { + fileInfo.append(k, v, true); + } + + /** + * Sets the file info offset in the trailer, finishes up populating fields in + * the file info, and writes the file info into the given data output. The + * reason the data output is not always {@link #outputStream} is that we store + * file info as a block in version 2. + * + * @param trailer fixed file trailer + * @param out the data output to write the file info to + * @throws IOException + */ + protected final void writeFileInfo(FixedFileTrailer trailer, DataOutput out) + throws IOException { + trailer.setFileInfoOffset(outputStream.getPos()); + finishFileInfo(); + fileInfo.write(out); + } + + /** + * Checks that the given key does not violate the key order. + * + * @param key Key to check. + * @return true if the key is duplicate + * @throws IOException if the key or the key order is wrong + */ + protected boolean checkKey(final byte[] key, final int offset, + final int length) throws IOException { + boolean isDuplicateKey = false; + + if (key == null || length <= 0) { + throw new IOException("Key cannot be null or empty"); + } + if (length > HFile.MAXIMUM_KEY_LENGTH) { + throw new IOException("Key length " + length + " > " + + HFile.MAXIMUM_KEY_LENGTH); + } + if (lastKeyBuffer != null) { + int keyComp = comparator.compare(lastKeyBuffer, lastKeyOffset, + lastKeyLength, key, offset, length); + if (keyComp > 0) { + throw new IOException("Added a key not lexically larger than" + + " previous key=" + + Bytes.toStringBinary(key, offset, length) + + ", lastkey=" + + Bytes.toStringBinary(lastKeyBuffer, lastKeyOffset, + lastKeyLength)); + } else if (keyComp == 0) { + isDuplicateKey = true; + } + } + return isDuplicateKey; + } + + /** Checks the given value for validity. */ + protected void checkValue(final byte[] value, final int offset, + final int length) throws IOException { + if (value == null) { + throw new IOException("Value cannot be null"); + } + } + + /** + * @return Path or null if we were passed a stream rather than a Path. + */ + @Override + public Path getPath() { + return path; + } + + @Override + public String toString() { + return "writer=" + (path != null ? path.toString() : null) + ", name=" + + name + ", compression=" + compressAlgo.getName(); + } + + /** + * Sets remaining trailer fields, writes the trailer to disk, and optionally + * closes the output stream. + */ + protected void finishClose(FixedFileTrailer trailer) throws IOException { + trailer.setMetaIndexCount(metaNames.size()); + trailer.setTotalUncompressedBytes(totalUncompressedBytes); + trailer.setEntryCount(entryCount); + trailer.setCompressionCodec(compressAlgo); + + trailer.serialize(outputStream); + + if (closeOutputStream) { + outputStream.close(); + outputStream = null; + } + } + + public static Compression.Algorithm compressionByName(String algoName) { + if (algoName == null) + return HFile.DEFAULT_COMPRESSION_ALGORITHM; + return Compression.getCompressionAlgorithmByName(algoName); + } + + /** A helper method to create HFile output streams in constructors */ + protected static FSDataOutputStream createOutputStream(Configuration conf, + FileSystem fs, Path path) throws IOException { + return fs.create(path, FsPermission.getDefault(), true, + fs.getConf().getInt("io.file.buffer.size", 4096), + fs.getDefaultReplication(), fs.getDefaultBlockSize(), + null); + } + + /** Initializes the block cache to use for cache-on-write */ + protected void initBlockCache() { + if (blockCache == null) { + blockCache = StoreFile.getBlockCache(conf); + conf = null; // This is all we need configuration for. + } + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java index 509121d..2d4002c 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java @@ -19,12 +19,15 @@ */ package org.apache.hadoop.hbase.io.hfile; -import java.nio.ByteBuffer; - +import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.LruBlockCache.CacheStats; /** - * Block cache interface. + * Block cache interface. Anything that implements the {@link HeapSize} + * interface can be put in the cache, because item size is all the cache + * cares about. We might move to a more specialized "cacheable" interface + * in the future. + * * TODO: Add filename or hash of filename to block cache key. */ public interface BlockCache { @@ -34,14 +37,14 @@ public interface BlockCache { * @param buf The block contents wrapped in a ByteBuffer. * @param inMemory Whether block should be treated as in-memory */ - public void cacheBlock(String blockName, ByteBuffer buf, boolean inMemory); + public void cacheBlock(String blockName, HeapSize buf, boolean inMemory); /** * Add block to cache (defaults to not in-memory). * @param blockName Zero-based file block number. * @param buf The block contents wrapped in a ByteBuffer. */ - public void cacheBlock(String blockName, ByteBuffer buf); + public void cacheBlock(String blockName, HeapSize buf); /** * Fetch block from cache. @@ -49,7 +52,7 @@ public interface BlockCache { * @param caching Whether this request has caching enabled (used for stats) * @return Block or null if block is not in the cache. */ - public ByteBuffer getBlock(String blockName, boolean caching); + public HeapSize getBlock(String blockName, boolean caching); /** * Evict block from cache. @@ -59,6 +62,17 @@ public interface BlockCache { public boolean evictBlock(String blockName); /** + * Evicts all blocks with name starting with the given prefix. This is + * necessary in cases we need to evict all blocks that belong to a particular + * HFile. In HFile v2 all blocks consist of the storefile name (UUID), an + * underscore, and the block offset in the file. An efficient implementation + * would avoid scanning all blocks in the cache. + * + * @return the number of blocks evicted + */ + public int evictBlocksByPrefix(String string); + + /** * Get the statistics for this block cache. * @return */ diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java new file mode 100644 index 0000000..59ef3ae --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java @@ -0,0 +1,168 @@ +/* + * Copyright 2011 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.DataInputStream; +import java.io.DataOutput; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Various types of {@link HFile} blocks. Ordinal values of these enum constants + * must not be relied upon. The values in the enum appear in the order they + * appear in a version 2 {@link HFile}. + */ +public enum BlockType { + + // Scanned block section + + /** Data block, both versions */ + DATA("DATABLK*"), + + /** Version 2 leaf index block. Appears in the data block section */ + LEAF_INDEX("IDXLEAF2"), + + /** Bloom filter block, version 2 */ + BLOOM_CHUNK("BLMFBLK2"), + + // Non-scanned block section + + /** Meta blocks */ + META("METABLKc"), + + /** Intermediate-level version 2 index in the non-data block section */ + INTERMEDIATE_INDEX("IDXINTE2"), + + // Load-on-open section. + + /** Root index block, also used for the single-level meta index, version 2 */ + ROOT_INDEX("IDXROOT2"), + + /** File info, version 2 */ + FILE_INFO("FILEINF2"), + + /** Bloom filter metadata, version 2 */ + BLOOM_META("BLMFMET2"), + + // Trailer + + /** Fixed file trailer, both versions (always just a magic string) */ + TRAILER("TRABLK\"$"), + + // Legacy blocks + + /** Block index magic string in version 1 */ + INDEX_V1("IDXBLK)+"); + + public static final int MAGIC_LENGTH = 8; + + private final byte[] magic; + + private BlockType(String magicStr) { + magic = Bytes.toBytes(magicStr); + assert magic.length == MAGIC_LENGTH; + } + + public void writeToStream(OutputStream out) throws IOException { + out.write(magic); + } + + public void write(DataOutput out) throws IOException { + out.write(magic); + } + + public void write(ByteBuffer buf) { + buf.put(magic); + } + + public static BlockType parse(byte[] buf, int offset, int length) + throws IOException { + if (length != MAGIC_LENGTH) { + throw new IOException("Magic record of invalid length: " + + Bytes.toStringBinary(buf, offset, length)); + } + + for (BlockType blockType : values()) + if (Bytes.compareTo(blockType.magic, 0, MAGIC_LENGTH, buf, offset, + MAGIC_LENGTH) == 0) + return blockType; + + throw new IOException("Invalid HFile block magic: " + + Bytes.toStringBinary(buf, offset, MAGIC_LENGTH)); + } + + public static BlockType read(DataInputStream in) throws IOException { + byte[] buf = new byte[MAGIC_LENGTH]; + in.readFully(buf); + return parse(buf, 0, buf.length); + } + + public static BlockType read(ByteBuffer buf) throws IOException { + BlockType blockType = parse(buf.array(), + buf.arrayOffset() + buf.position(), + Math.min(buf.limit() - buf.position(), MAGIC_LENGTH)); + + // If we got here, we have read exactly MAGIC_LENGTH bytes. + buf.position(buf.position() + MAGIC_LENGTH); + return blockType; + } + + /** + * Put the magic record out to the specified byte array position. + * + * @param bytes the byte array + * @param offset position in the array + * @return incremented offset + */ + public int put(byte[] bytes, int offset) { + System.arraycopy(magic, 0, bytes, offset, MAGIC_LENGTH); + return offset + MAGIC_LENGTH; + } + + /** + * Reads a magic record of the length {@link #MAGIC_LENGTH} from the given + * stream and expects it to match this block type. + */ + public void readAndCheck(DataInputStream in) throws IOException { + byte[] buf = new byte[MAGIC_LENGTH]; + in.readFully(buf); + if (Bytes.compareTo(buf, magic) != 0) { + throw new IOException("Invalid magic: expected " + + Bytes.toStringBinary(magic) + ", got " + Bytes.toStringBinary(buf)); + } + } + + /** + * Reads a magic record of the length {@link #MAGIC_LENGTH} from the given + * byte buffer and expects it to match this block type. + */ + public void readAndCheck(ByteBuffer in) throws IOException { + byte[] buf = new byte[MAGIC_LENGTH]; + in.get(buf); + if (Bytes.compareTo(buf, magic) != 0) { + throw new IOException("Invalid magic: expected " + + Bytes.toStringBinary(magic) + ", got " + Bytes.toStringBinary(buf)); + } + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/CachedBlock.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/CachedBlock.java index fa0a79d..3b130d8 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/CachedBlock.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/CachedBlock.java @@ -55,22 +55,22 @@ public class CachedBlock implements HeapSize, Comparable { }; private final String blockName; - private final ByteBuffer buf; + private final HeapSize buf; private volatile long accessTime; private long size; private BlockPriority priority; - public CachedBlock(String blockName, ByteBuffer buf, long accessTime) { + public CachedBlock(String blockName, HeapSize buf, long accessTime) { this(blockName, buf, accessTime, false); } - public CachedBlock(String blockName, ByteBuffer buf, long accessTime, + public CachedBlock(String blockName, HeapSize buf, long accessTime, boolean inMemory) { this.blockName = blockName; this.buf = buf; this.accessTime = accessTime; - this.size = ClassSize.align(blockName.length()) + - ClassSize.align(buf.capacity()) + PER_BLOCK_OVERHEAD; + this.size = ClassSize.align(blockName.length()) + + ClassSize.align(buf.heapSize()) + PER_BLOCK_OVERHEAD; if(inMemory) { this.priority = BlockPriority.MEMORY; } else { @@ -97,7 +97,7 @@ public class CachedBlock implements HeapSize, Comparable { return this.accessTime < that.accessTime ? 1 : -1; } - public ByteBuffer getBuffer() { + public HeapSize getBuffer() { return this.buf; } diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java new file mode 100644 index 0000000..a3a3b91 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java @@ -0,0 +1,475 @@ +/* + * Copyright 2011 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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +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.FSDataInputStream; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.RawComparator; + +import static org.apache.hadoop.hbase.io.hfile.HFile.MIN_FORMAT_VERSION; +import static org.apache.hadoop.hbase.io.hfile.HFile.MAX_FORMAT_VERSION; + +import com.google.common.io.NullOutputStream; + +/** + * The {@link HFile} has a fixed trailer which contains offsets to other + * variable parts of the file. Also includes basic metadata on this file. The + * trailer size is fixed within a given {@link HFile} format version only, but + * we always store the version number as the last four-byte integer of the file. + */ +public class FixedFileTrailer { + + private static final Log LOG = LogFactory.getLog(FixedFileTrailer.class); + + /** + * We store the comparator class name as a fixed-length field in the trailer. + */ + private static final int MAX_COMPARATOR_NAME_LENGTH = 128; + + /** + * Offset to the fileinfo data, a small block of vitals. Necessary in v1 but + * only potentially useful for pretty-printing in v2. + */ + private long fileInfoOffset; + + /** + * In version 1, the offset to the data block index. Starting from version 2, + * the meaning of this field is the offset to the section of the file that + * should be loaded at the time the file is being opened, and as of the time + * of writing, this happens to be the offset of the file info section. + */ + private long loadOnOpenDataOffset; + + /** The number of entries in the root data index. */ + private int dataIndexCount; + + /** Total uncompressed size of all blocks of the data index */ + private long uncompressedDataIndexSize; + + /** The number of entries in the meta index */ + private int metaIndexCount; + + /** The total uncompressed size of keys/values stored in the file. */ + private long totalUncompressedBytes; + + /** + * The number of key/value pairs in the file. This field was int in version 1, + * but is now long. + */ + private long entryCount; + + /** The compression codec used for all blocks. */ + private Compression.Algorithm compressionCodec = Compression.Algorithm.NONE; + + /** + * The number of levels in the potentially multi-level data index. Used from + * version 2 onwards. + */ + private int numDataIndexLevels; + + /** The offset of the first data block. */ + private long firstDataBlockOffset; + + /** + * It is guaranteed that no key/value data blocks start after this offset in + * the file. + */ + private long lastDataBlockOffset; + + /** Raw key comparator class name in version 2 */ + private String comparatorClassName = RawComparator.class.getName(); + + /** The {@link HFile} format version. */ + private final int version; + + FixedFileTrailer(int version) { + this.version = version; + HFile.checkFormatVersion(version); + } + + private static int[] computeTrailerSizeByVersion() { + int versionToSize[] = new int[HFile.MAX_FORMAT_VERSION + 1]; + for (int version = MIN_FORMAT_VERSION; + version <= MAX_FORMAT_VERSION; + ++version) { + FixedFileTrailer fft = new FixedFileTrailer(version); + DataOutputStream dos = new DataOutputStream(new NullOutputStream()); + try { + fft.serialize(dos); + } catch (IOException ex) { + // The above has no reason to fail. + throw new RuntimeException(ex); + } + versionToSize[version] = dos.size(); + } + return versionToSize; + } + + private static int getMaxTrailerSize() { + int maxSize = 0; + for (int version = MIN_FORMAT_VERSION; + version <= MAX_FORMAT_VERSION; + ++version) + maxSize = Math.max(getTrailerSize(version), maxSize); + return maxSize; + } + + private static final int TRAILER_SIZE[] = computeTrailerSizeByVersion(); + private static final int MAX_TRAILER_SIZE = getMaxTrailerSize(); + + static int getTrailerSize(int version) { + return TRAILER_SIZE[version]; + } + + public int getTrailerSize() { + return getTrailerSize(version); + } + + /** + * Write the trailer to a data stream. We support writing version 1 for + * testing and for determining version 1 trailer size. It is also easy to see + * what fields changed in version 2. + * + * @param outputStream + * @throws IOException + */ + void serialize(DataOutputStream outputStream) throws IOException { + HFile.checkFormatVersion(version); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutput baosDos = new DataOutputStream(baos); + + BlockType.TRAILER.write(baosDos); + baosDos.writeLong(fileInfoOffset); + baosDos.writeLong(loadOnOpenDataOffset); + baosDos.writeInt(dataIndexCount); + + if (version == 1) { + // This used to be metaIndexOffset, but it was not used in version 1. + baosDos.writeLong(0); + } else { + baosDos.writeLong(uncompressedDataIndexSize); + } + + baosDos.writeInt(metaIndexCount); + baosDos.writeLong(totalUncompressedBytes); + if (version == 1) { + baosDos.writeInt((int) Math.min(Integer.MAX_VALUE, entryCount)); + } else { + // This field is long from version 2 onwards. + baosDos.writeLong(entryCount); + } + baosDos.writeInt(compressionCodec.ordinal()); + + if (version > 1) { + baosDos.writeInt(numDataIndexLevels); + baosDos.writeLong(firstDataBlockOffset); + baosDos.writeLong(lastDataBlockOffset); + Bytes.writeStringFixedSize(baosDos, comparatorClassName, + MAX_COMPARATOR_NAME_LENGTH); + } + baosDos.writeInt(version); + + outputStream.write(baos.toByteArray()); + } + + /** + * Deserialize the fixed file trailer from the given stream. The version needs + * to already be specified. Make sure this is consistent with + * {@link #serialize(DataOutputStream)}. + * + * @param inputStream + * @param version + * @throws IOException + */ + void deserialize(DataInputStream inputStream) throws IOException { + HFile.checkFormatVersion(version); + + BlockType.TRAILER.readAndCheck(inputStream); + + fileInfoOffset = inputStream.readLong(); + loadOnOpenDataOffset = inputStream.readLong(); + dataIndexCount = inputStream.readInt(); + + if (version == 1) { + inputStream.readLong(); // Read and skip metaIndexOffset. + } else { + uncompressedDataIndexSize = inputStream.readLong(); + } + metaIndexCount = inputStream.readInt(); + + totalUncompressedBytes = inputStream.readLong(); + entryCount = version == 1 ? inputStream.readInt() : inputStream.readLong(); + compressionCodec = Compression.Algorithm.values()[inputStream.readInt()]; + if (version > 1) { + numDataIndexLevels = inputStream.readInt(); + firstDataBlockOffset = inputStream.readLong(); + lastDataBlockOffset = inputStream.readLong(); + comparatorClassName = + Bytes.readStringFixedSize(inputStream, MAX_COMPARATOR_NAME_LENGTH); + } + + expectVersion(inputStream.readInt()); + } + + private void append(StringBuilder sb, String s) { + if (sb.length() > 0) + sb.append(", "); + sb.append(s); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + append(sb, "fileinfoOffset=" + fileInfoOffset); + append(sb, "loadOnOpenDataOffset=" + loadOnOpenDataOffset); + append(sb, "dataIndexCount=" + dataIndexCount); + append(sb, "metaIndexCount=" + metaIndexCount); + append(sb, "totalUncomressedBytes=" + totalUncompressedBytes); + append(sb, "entryCount=" + entryCount); + append(sb, "compressionCodec=" + compressionCodec); + if (version == 2) { + append(sb, "uncompressedDataIndexSize=" + uncompressedDataIndexSize); + append(sb, "numDataIndexLevels=" + numDataIndexLevels); + append(sb, "firstDataBlockOffset=" + firstDataBlockOffset); + append(sb, "lastDataBlockOffset=" + lastDataBlockOffset); + append(sb, "comparatorClassName=" + comparatorClassName); + } + append(sb, "version=" + version); + + return sb.toString(); + } + + /** + * Reads a file trailer from the given file. + * + * @param istream the input stream with the ability to seek. Does not have to + * be buffered, as only one read operation is made. + * @param fileSize the file size. Can be obtained using + * {@link org.apache.hadoop.fs.FileSystem#getFileStatus( + * org.apache.hadoop.fs.Path)}. + * @return the fixed file trailer read + * @throws IOException if failed to read from the underlying stream, or the + * trailer is corrupted, or the version of the trailer is + * unsupported + */ + public static FixedFileTrailer readFromStream(FSDataInputStream istream, + long fileSize) throws IOException { + int bufferSize = MAX_TRAILER_SIZE; + long seekPoint = fileSize - bufferSize; + if (seekPoint < 0) { + // It is hard to imagine such a small HFile. + seekPoint = 0; + bufferSize = (int) fileSize; + } + + istream.seek(seekPoint); + ByteBuffer buf = ByteBuffer.allocate(bufferSize); + istream.readFully(buf.array(), buf.arrayOffset(), + buf.arrayOffset() + buf.limit()); + + // Read the version from the last int of the file. + buf.position(buf.limit() - Bytes.SIZEOF_INT); + int version = buf.getInt(); + + try { + HFile.checkFormatVersion(version); + } catch (IllegalArgumentException iae) { + // In this context, an invalid version might indicate a corrupt HFile. + throw new IOException(iae); + } + + int trailerSize = getTrailerSize(version); + + FixedFileTrailer fft = new FixedFileTrailer(version); + fft.deserialize(new DataInputStream(new ByteArrayInputStream(buf.array(), + buf.arrayOffset() + bufferSize - trailerSize, trailerSize))); + return fft; + } + + public void expectVersion(int expected) { + if (version != expected) { + throw new IllegalArgumentException("Invalid HFile version: " + version + + " (expected: " + expected + ")"); + } + } + + public void expectAtLeastVersion(int lowerBound) { + if (version < lowerBound) { + throw new IllegalArgumentException("Invalid HFile version: " + version + + " (expected: " + lowerBound + " or higher)."); + } + } + + public long getFileInfoOffset() { + return fileInfoOffset; + } + + public void setFileInfoOffset(long fileInfoOffset) { + this.fileInfoOffset = fileInfoOffset; + } + + public long getLoadOnOpenDataOffset() { + return loadOnOpenDataOffset; + } + + public void setLoadOnOpenOffset(long loadOnOpenDataOffset) { + this.loadOnOpenDataOffset = loadOnOpenDataOffset; + } + + public int getDataIndexCount() { + return dataIndexCount; + } + + public void setDataIndexCount(int dataIndexCount) { + this.dataIndexCount = dataIndexCount; + } + + public int getMetaIndexCount() { + return metaIndexCount; + } + + public void setMetaIndexCount(int metaIndexCount) { + this.metaIndexCount = metaIndexCount; + } + + public long getTotalUncompressedBytes() { + return totalUncompressedBytes; + } + + public void setTotalUncompressedBytes(long totalUncompressedBytes) { + this.totalUncompressedBytes = totalUncompressedBytes; + } + + public long getEntryCount() { + return entryCount; + } + + public void setEntryCount(long newEntryCount) { + if (version == 1) { + int intEntryCount = (int) Math.min(Integer.MAX_VALUE, newEntryCount); + if (intEntryCount != newEntryCount) { + LOG.info("Warning: entry count is " + newEntryCount + " but writing " + + intEntryCount + " into the version " + version + " trailer"); + } + entryCount = intEntryCount; + return; + } + entryCount = newEntryCount; + } + + public Compression.Algorithm getCompressionCodec() { + return compressionCodec; + } + + public void setCompressionCodec(Compression.Algorithm compressionCodec) { + this.compressionCodec = compressionCodec; + } + + public int getNumDataIndexLevels() { + expectAtLeastVersion(2); + return numDataIndexLevels; + } + + public void setNumDataIndexLevels(int numDataIndexLevels) { + expectAtLeastVersion(2); + this.numDataIndexLevels = numDataIndexLevels; + } + + public long getLastDataBlockOffset() { + expectAtLeastVersion(2); + return lastDataBlockOffset; + } + + public void setLastDataBlockOffset(long lastDataBlockOffset) { + expectAtLeastVersion(2); + this.lastDataBlockOffset = lastDataBlockOffset; + } + + public long getFirstDataBlockOffset() { + expectAtLeastVersion(2); + return firstDataBlockOffset; + } + + public void setFirstDataBlockOffset(long firstDataBlockOffset) { + expectAtLeastVersion(2); + this.firstDataBlockOffset = firstDataBlockOffset; + } + + public int getVersion() { + return version; + } + + @SuppressWarnings("rawtypes") + public void setComparatorClass(Class klass) { + expectAtLeastVersion(2); + comparatorClassName = klass.getName(); + } + + @SuppressWarnings("unchecked") + private static Class> getComparatorClass( + String comparatorClassName) throws IOException { + try { + return (Class>) + Class.forName(comparatorClassName); + } catch (ClassNotFoundException ex) { + throw new IOException(ex); + } + } + + public static RawComparator createComparator( + String comparatorClassName) throws IOException { + try { + return getComparatorClass(comparatorClassName).newInstance(); + } catch (InstantiationException e) { + throw new IOException(e); + } catch (IllegalAccessException e) { + throw new IOException(e); + } + } + + RawComparator createComparator() throws IOException { + expectAtLeastVersion(2); + return createComparator(comparatorClassName); + } + + public long getUncompressedDataIndexSize() { + if (version == 1) + return 0; + return uncompressedDataIndexSize; + } + + public void setUncompressedDataIndexSize( + long uncompressedDataIndexSize) { + expectAtLeastVersion(2); + this.uncompressedDataIndexSize = uncompressedDataIndexSize; + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 5963552..a3d778e 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -19,26 +19,15 @@ */ package org.apache.hadoop.hbase.io.hfile; -import java.io.BufferedInputStream; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.Closeable; -import java.io.DataInputStream; -import java.io.DataOutputStream; +import java.io.DataInput; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.HelpFormatter; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.PosixParser; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -48,26 +37,14 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.hbase.KeyValue.KeyComparator; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.KeyComparator; import org.apache.hadoop.hbase.io.HbaseMapWritable; -import org.apache.hadoop.hbase.io.HeapSize; -import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; -import org.apache.hadoop.hbase.util.BloomFilter; -import org.apache.hadoop.hbase.util.ByteBloomFilter; -import org.apache.hadoop.hbase.util.ByteBufferOutputStream; +import org.apache.hadoop.hbase.util.BloomFilterWriter; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hadoop.hbase.util.CompressionTest; import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.compress.Compressor; -import org.apache.hadoop.io.compress.Decompressor; /** * File format for hbase. @@ -135,25 +112,13 @@ import org.apache.hadoop.io.compress.Decompressor; public class HFile { static final Log LOG = LogFactory.getLog(HFile.class); - /* These values are more or less arbitrary, and they are used as a - * form of check to make sure the file isn't completely corrupt. - */ - final static byte [] DATABLOCKMAGIC = - {'D', 'A', 'T', 'A', 'B', 'L', 'K', 42 }; - final static byte [] INDEXBLOCKMAGIC = - { 'I', 'D', 'X', 'B', 'L', 'K', 41, 43 }; - final static byte [] METABLOCKMAGIC = - { 'M', 'E', 'T', 'A', 'B', 'L', 'K', 99 }; - final static byte [] TRAILERBLOCKMAGIC = - { 'T', 'R', 'A', 'B', 'L', 'K', 34, 36 }; - /** * Maximum length of key in HFile. */ public final static int MAXIMUM_KEY_LENGTH = Integer.MAX_VALUE; /** - * Default blocksize for hfile. + * Default block size for an HFile. */ public final static int DEFAULT_BLOCKSIZE = 64 * 1024; @@ -162,1653 +127,229 @@ public class HFile { */ public final static Compression.Algorithm DEFAULT_COMPRESSION_ALGORITHM = Compression.Algorithm.NONE; + + /** Minimum supported HFile format version */ + public static final int MIN_FORMAT_VERSION = 1; + + /** Maximum supported HFile format version */ + public static final int MAX_FORMAT_VERSION = 2; + /** Default compression name: none. */ public final static String DEFAULT_COMPRESSION = DEFAULT_COMPRESSION_ALGORITHM.getName(); + /** Separator between HFile name and offset in block cache key */ + static final char CACHE_KEY_SEPARATOR = '_'; + // For measuring latency of "typical" reads and writes - private static volatile long readOps; - private static volatile long readTime; - private static volatile long writeOps; - private static volatile long writeTime; + static volatile AtomicLong readOps = new AtomicLong(); + static volatile AtomicLong readTimeNano = new AtomicLong(); + static volatile AtomicLong writeOps = new AtomicLong(); + static volatile AtomicLong writeTimeNano = new AtomicLong(); public static final long getReadOps() { - long ret = readOps; - readOps = 0; - return ret; + return readOps.getAndSet(0); } - public static final long getReadTime() { - long ret = readTime; - readTime = 0; - return ret; + public static final long getReadTimeMs() { + return readTimeNano.getAndSet(0) / 1000000; } public static final long getWriteOps() { - long ret = writeOps; - writeOps = 0; - return ret; + return writeOps.getAndSet(0); } - public static final long getWriteTime() { - long ret = writeTime; - writeTime = 0; - return ret; + public static final long getWriteTimeMs() { + return writeTimeNano.getAndSet(0) / 1000000; } - /** - * HFile Writer. - */ - public static class Writer implements Closeable { - // FileSystem stream to write on. - private FSDataOutputStream outputStream; - // True if we opened the outputStream (and so will close it). - private boolean closeOutputStream; - - // Name for this object used when logging or in toString. Is either - // the result of a toString on stream or else toString of passed file Path. - protected String name; - - // Total uncompressed bytes, maybe calculate a compression ratio later. - private long totalBytes = 0; - - // Total # of key/value entries, ie: how many times add() was called. - private int entryCount = 0; + /** API required to write an {@link HFile} */ + public interface Writer extends Closeable { - // Used calculating average key and value lengths. - private long keylength = 0; - private long valuelength = 0; + /** Add an element to the file info map. */ + void appendFileInfo(byte[] key, byte[] value) throws IOException; - // Used to ensure we write in order. - private final RawComparator comparator; + void append(KeyValue kv) throws IOException; - // Number of uncompressed bytes per block. Reinitialized when we start - // new block. - private int blocksize; - - // Offset where the current block began. - private long blockBegin; - - // First key in a block (Not first key in file). - private byte [] firstKey = null; - - // Key previously appended. Becomes the last key in the file. - private byte [] lastKeyBuffer = null; - private int lastKeyOffset = -1; - private int lastKeyLength = -1; - - // See {@link BlockIndex}. Below four fields are used to write the block - // index. - ArrayList blockKeys = new ArrayList(); - // Block offset in backing stream. - ArrayList blockOffsets = new ArrayList(); - // Raw (decompressed) data size. - ArrayList blockDataSizes = new ArrayList(); - - // Meta block system. - private ArrayList metaNames = new ArrayList(); - private ArrayList metaData = new ArrayList(); - - // Used compression. Used even if no compression -- 'none'. - private final Compression.Algorithm compressAlgo; - private Compressor compressor; - - // Special datastructure to hold fileinfo. - private FileInfo fileinfo = new FileInfo(); - - // May be null if we were passed a stream. - private Path path = null; - - // Block cache to optionally fill on write - private BlockCache blockCache; - - // Byte buffer output stream made per block written. - private ByteBufferOutputStream bbos = null; - private DataOutputStream bbosDos = null; - private int blockNumber = 0; - - /** - * Constructor that uses all defaults for compression and block size. - * @param fs - * @param path - * @throws IOException - */ - public Writer(FileSystem fs, Path path) - throws IOException { - this(fs, path, DEFAULT_BLOCKSIZE, (Compression.Algorithm) null, null, - null); - } + void append(byte[] key, byte[] value) throws IOException; - /** - * Constructor that takes a Path. - * @param fs - * @param path - * @param blocksize - * @param compress - * @param comparator - * @throws IOException - * @throws IOException - */ - public Writer(FileSystem fs, Path path, int blocksize, - String compress, final KeyComparator comparator) - throws IOException { - this(fs, path, blocksize, - compress == null? DEFAULT_COMPRESSION_ALGORITHM: - Compression.getCompressionAlgorithmByName(compress), - comparator, null); - } + /** @return the path to this {@link HFile} */ + Path getPath(); - /** - * Constructor that takes a Path. - * @param fs - * @param path - * @param blocksize - * @param compress - * @param comparator - * @throws IOException - */ - public Writer(FileSystem fs, Path path, int blocksize, - Compression.Algorithm compress, - final KeyComparator comparator, BlockCache blockCache) - throws IOException { - this(fs.create(path), blocksize, compress, comparator); - this.closeOutputStream = true; - this.name = path.toString(); - this.path = path; - this.blockCache = blockCache; - } + void appendMetaBlock(String bloomFilterMetaKey, Writable metaWriter); /** - * Constructor that takes a stream. - * @param ostream Stream to use. - * @param blocksize - * @param compress - * @param c RawComparator to use. - * @throws IOException + * Adds an inline block writer such as a multi-level block index writer or + * a compound Bloom filter writer. */ - public Writer(final FSDataOutputStream ostream, final int blocksize, - final String compress, final KeyComparator c) - throws IOException { - this(ostream, blocksize, - Compression.getCompressionAlgorithmByName(compress), c); - } + void addInlineBlockWriter(InlineBlockWriter bloomWriter); /** - * Constructor that takes a stream. - * @param ostream Stream to use. - * @param blocksize - * @param compress - * @param c - * @throws IOException - */ - public Writer(final FSDataOutputStream ostream, final int blocksize, - final Compression.Algorithm compress, final KeyComparator c) - throws IOException { - this.outputStream = ostream; - this.closeOutputStream = false; - this.blocksize = blocksize; - this.comparator = c == null? Bytes.BYTES_RAWCOMPARATOR: c; - this.name = this.outputStream.toString(); - this.compressAlgo = compress == null? - DEFAULT_COMPRESSION_ALGORITHM: compress; - } - - /* - * If at block boundary, opens new block. - * @throws IOException + * Store Bloom filter in the file. This does not deal with Bloom filter + * internals but is necessary, since Bloom filters are stored differently + * in HFile version 1 and version 2. */ - private void checkBlockBoundary() throws IOException { - if (bbosDos != null && bbosDos.size() < blocksize) return; - finishBlock(); - newBlock(); - } - - /* - * Do the cleanup if a current block. - * @throws IOException - */ - private void finishBlock() throws IOException { - if (bbosDos == null) return; - - // Flush Data Output Stream - bbosDos.flush(); - - // Compress Data and write to output stream - DataOutputStream compressStream = getCompressingStream(); - bbos.writeTo(compressStream); - int size = releaseCompressingStream(compressStream); - - long now = System.currentTimeMillis(); - - blockKeys.add(firstKey); - blockOffsets.add(Long.valueOf(blockBegin)); - blockDataSizes.add(Integer.valueOf(size)); - this.totalBytes += size; - - writeTime += System.currentTimeMillis() - now; - writeOps++; - - if (blockCache != null) { - byte[] bytes = bbos.toByteArray(DATABLOCKMAGIC.length, bbos.size() - DATABLOCKMAGIC.length); - ByteBuffer blockToCache = ByteBuffer.wrap(bytes); - String blockName = path.toString() + blockNumber; - blockCache.cacheBlock(blockName, blockToCache); - } - - bbosDos.close(); - bbosDos = null; - bbos = null; - - blockNumber++; - } - - /* - * Ready a new block for writing. - * @throws IOException - */ - private void newBlock() throws IOException { - // This is where the next block begins. - blockBegin = outputStream.getPos(); - - firstKey = null; - - // to avoid too many calls to realloc(), - // pre-allocates the byte stream to the block size + 25% - // only if blocksize is under 1Gb - int bbosBlocksize = Math.max(blocksize, blocksize + (blocksize / 4)); - bbos = new ByteBufferOutputStream(bbosBlocksize); - bbosDos = new DataOutputStream(bbos); - bbosDos.write(DATABLOCKMAGIC); - } - - /* - * Sets up a compressor and creates a compression stream on top of - * this.outputStream. Get one per block written. - * @return A compressing stream; if 'none' compression, returned stream - * does not compress. - * @throws IOException - * @see {@link #releaseCompressingStream(DataOutputStream)} - */ - private DataOutputStream getCompressingStream() throws IOException { - this.compressor = compressAlgo.getCompressor(); - // Get new DOS compression stream. In tfile, the DOS, is not closed, - // just finished, and that seems to be fine over there. TODO: Check - // no memory retention of the DOS. Should I disable the 'flush' on the - // DOS as the BCFile over in tfile does? It wants to make it so flushes - // don't go through to the underlying compressed stream. Flush on the - // compressed downstream should be only when done. I was going to but - // looks like when we call flush in here, its legitimate flush that - // should go through to the compressor. - OutputStream os = - this.compressAlgo.createCompressionStream(this.outputStream, - this.compressor, 0); - return new DataOutputStream(os); - } - - /* - * Let go of block compressor and compressing stream gotten in call - * {@link #getCompressingStream}. - * @param dos - * @return How much was written on this stream since it was taken out. - * @see #getCompressingStream() - * @throws IOException - */ - private int releaseCompressingStream(final DataOutputStream dos) - throws IOException { - dos.flush(); - this.compressAlgo.returnCompressor(this.compressor); - this.compressor = null; - return dos.size(); - } - - /** - * Add a meta block to the end of the file. Call before close(). - * Metadata blocks are expensive. Fill one with a bunch of serialized data - * rather than do a metadata block per metadata instance. If metadata is - * small, consider adding to file info using - * {@link #appendFileInfo(byte[], byte[])} - * @param metaBlockName name of the block - * @param content will call readFields to get data later (DO NOT REUSE) - */ - public void appendMetaBlock(String metaBlockName, Writable content) { - byte[] key = Bytes.toBytes(metaBlockName); - int i; - for (i = 0; i < metaNames.size(); ++i) { - // stop when the current key is greater than our own - byte[] cur = metaNames.get(i); - if (Bytes.BYTES_RAWCOMPARATOR.compare(cur, 0, cur.length, - key, 0, key.length) > 0) { - break; - } - } - metaNames.add(i, key); - metaData.add(i, content); - } - - /** - * Add to the file info. Added key value can be gotten out of the return - * from {@link Reader#loadFileInfo()}. - * @param k Key - * @param v Value - * @throws IOException - */ - public void appendFileInfo(final byte [] k, final byte [] v) - throws IOException { - appendFileInfo(this.fileinfo, k, v, true); - } - - static FileInfo appendFileInfo(FileInfo fi, final byte [] k, final byte [] v, - final boolean checkPrefix) - throws IOException { - if (k == null || v == null) { - throw new NullPointerException("Key nor value may be null"); - } - if (checkPrefix && - Bytes.startsWith(k, FileInfo.RESERVED_PREFIX_BYTES)) { - throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX + - " are reserved"); - } - fi.put(k, v); - return fi; - } - - /** - * @return Path or null if we were passed a stream rather than a Path. - */ - public Path getPath() { - return this.path; - } - - @Override - public String toString() { - return "writer=" + this.name + ", compression=" + - this.compressAlgo.getName(); - } - - /** - * Add key/value to file. - * Keys must be added in an order that agrees with the Comparator passed - * on construction. - * @param kv KeyValue to add. Cannot be empty nor null. - * @throws IOException - */ - public void append(final KeyValue kv) - throws IOException { - append(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(), - kv.getBuffer(), kv.getValueOffset(), kv.getValueLength()); - } - - /** - * Add key/value to file. - * Keys must be added in an order that agrees with the Comparator passed - * on construction. - * @param key Key to add. Cannot be empty nor null. - * @param value Value to add. Cannot be empty nor null. - * @throws IOException - */ - public void append(final byte [] key, final byte [] value) - throws IOException { - append(key, 0, key.length, value, 0, value.length); - } - - /** - * Add key/value to file. - * Keys must be added in an order that agrees with the Comparator passed - * on construction. - * @param key - * @param koffset - * @param klength - * @param value - * @param voffset - * @param vlength - * @throws IOException - */ - private void append(final byte [] key, final int koffset, final int klength, - final byte [] value, final int voffset, final int vlength) - throws IOException { - boolean dupKey = checkKey(key, koffset, klength); - checkValue(value, voffset, vlength); - if (!dupKey) { - checkBlockBoundary(); - } - // Write length of key and value and then actual key and value bytes. - this.bbosDos.writeInt(klength); - this.keylength += klength; - this.bbosDos.writeInt(vlength); - this.valuelength += vlength; - this.bbosDos.write(key, koffset, klength); - this.bbosDos.write(value, voffset, vlength); - // Are we the first key in this block? - if (this.firstKey == null) { - // Copy the key. - this.firstKey = new byte [klength]; - System.arraycopy(key, koffset, this.firstKey, 0, klength); - } - this.lastKeyBuffer = key; - this.lastKeyOffset = koffset; - this.lastKeyLength = klength; - this.entryCount ++; - } - - /* - * @param key Key to check. - * @return the flag of duplicate Key or not - * @throws IOException - */ - private boolean checkKey(final byte [] key, final int offset, final int length) - throws IOException { - boolean dupKey = false; - - if (key == null || length <= 0) { - throw new IOException("Key cannot be null or empty"); - } - if (length > MAXIMUM_KEY_LENGTH) { - throw new IOException("Key length " + length + " > " + - MAXIMUM_KEY_LENGTH); - } - if (this.lastKeyBuffer != null) { - int keyComp = this.comparator.compare(this.lastKeyBuffer, this.lastKeyOffset, - this.lastKeyLength, key, offset, length); - if (keyComp > 0) { - throw new IOException("Added a key not lexically larger than" + - " previous key=" + Bytes.toStringBinary(key, offset, length) + - ", lastkey=" + Bytes.toStringBinary(this.lastKeyBuffer, this.lastKeyOffset, - this.lastKeyLength)); - } else if (keyComp == 0) { - dupKey = true; - } - } - return dupKey; - } - - private void checkValue(final byte [] value, final int offset, - final int length) throws IOException { - if (value == null) { - throw new IOException("Value cannot be null"); - } - } - - public long getTotalBytes() { - return this.totalBytes; - } - - public void close() throws IOException { - if (this.outputStream == null) { - return; - } - // Write out the end of the data blocks, then write meta data blocks. - // followed by fileinfo, data block index and meta block index. - - finishBlock(); - - FixedFileTrailer trailer = new FixedFileTrailer(); - - // Write out the metadata blocks if any. - ArrayList metaOffsets = null; - ArrayList metaDataSizes = null; - if (metaNames.size() > 0) { - metaOffsets = new ArrayList(metaNames.size()); - metaDataSizes = new ArrayList(metaNames.size()); - for (int i = 0 ; i < metaNames.size() ; ++ i ) { - // store the beginning offset - long curPos = outputStream.getPos(); - metaOffsets.add(curPos); - // write the metadata content - DataOutputStream dos = getCompressingStream(); - dos.write(METABLOCKMAGIC); - metaData.get(i).write(dos); - int size = releaseCompressingStream(dos); - // store the metadata size - metaDataSizes.add(size); - } - } + void addBloomFilter(BloomFilterWriter bfw); + } - // Write fileinfo. - trailer.fileinfoOffset = writeFileInfo(this.outputStream); + /** + * This variety of ways to construct writers is used throughout the code, and + * we want to be able to swap writer implementations. + */ + public static abstract class WriterFactory { + protected Configuration conf; - // Write the data block index. - trailer.dataIndexOffset = BlockIndex.writeIndex(this.outputStream, - this.blockKeys, this.blockOffsets, this.blockDataSizes); + WriterFactory(Configuration conf) { this.conf = conf; } - // Meta block index. - if (metaNames.size() > 0) { - trailer.metaIndexOffset = BlockIndex.writeIndex(this.outputStream, - this.metaNames, metaOffsets, metaDataSizes); - } + public abstract Writer createWriter(FileSystem fs, Path path) + throws IOException; - // Now finish off the trailer. - trailer.dataIndexCount = blockKeys.size(); - trailer.metaIndexCount = metaNames.size(); + public abstract Writer createWriter(FileSystem fs, Path path, + int blockSize, Compression.Algorithm compress, + final KeyComparator comparator) throws IOException; - trailer.totalUncompressedBytes = totalBytes; - trailer.entryCount = entryCount; + public abstract Writer createWriter(FileSystem fs, Path path, + int blockSize, String compress, + final KeyComparator comparator) throws IOException; - trailer.compressionCodec = this.compressAlgo.ordinal(); + public abstract Writer createWriter(final FSDataOutputStream ostream, + final int blockSize, final String compress, + final KeyComparator comparator) throws IOException; - trailer.serialize(outputStream); + public abstract Writer createWriter(final FSDataOutputStream ostream, + final int blockSize, final Compression.Algorithm compress, + final KeyComparator c) throws IOException; + } - if (this.closeOutputStream) { - this.outputStream.close(); - this.outputStream = null; - } - } + /** The configuration key for HFile version to use for new files */ + public static final String FORMAT_VERSION_KEY = "hfile.format.version"; - /* - * Add last bits of metadata to fileinfo and then write it out. - * Reader will be expecting to find all below. - * @param o Stream to write on. - * @return Position at which we started writing. - * @throws IOException - */ - private long writeFileInfo(FSDataOutputStream o) throws IOException { - if (this.lastKeyBuffer != null) { - // Make a copy. The copy is stuffed into HMapWritable. Needs a clean - // byte buffer. Won't take a tuple. - byte [] b = new byte[this.lastKeyLength]; - System.arraycopy(this.lastKeyBuffer, this.lastKeyOffset, b, 0, - this.lastKeyLength); - appendFileInfo(this.fileinfo, FileInfo.LASTKEY, b, false); - } - int avgKeyLen = this.entryCount == 0? 0: - (int)(this.keylength/this.entryCount); - appendFileInfo(this.fileinfo, FileInfo.AVG_KEY_LEN, - Bytes.toBytes(avgKeyLen), false); - int avgValueLen = this.entryCount == 0? 0: - (int)(this.valuelength/this.entryCount); - appendFileInfo(this.fileinfo, FileInfo.AVG_VALUE_LEN, - Bytes.toBytes(avgValueLen), false); - appendFileInfo(this.fileinfo, FileInfo.COMPARATOR, - Bytes.toBytes(this.comparator.getClass().getName()), false); - long pos = o.getPos(); - this.fileinfo.write(o); - return pos; - } + public static int getFormatVersion(Configuration conf) { + int version = conf.getInt(FORMAT_VERSION_KEY, MAX_FORMAT_VERSION); + checkFormatVersion(version); + return version; } /** - * HFile Reader. + * Returns the factory to be used to create {@link HFile} writers. Should + * always be {@link HFileWriterV2#WRITER_FACTORY_V2} in production, but + * can also be {@link HFileWriterV1#WRITER_FACTORY_V1} in testing. */ - public static class Reader implements Closeable { - // Stream to read from. - private FSDataInputStream istream; - // True if we should close istream when done. We don't close it if we - // didn't open it. - private boolean closeIStream; - - // These are read in when the file info is loaded. - HFile.BlockIndex blockIndex; - private BlockIndex metaIndex; - FixedFileTrailer trailer; - private volatile boolean fileInfoLoaded = false; - - // Filled when we read in the trailer. - private Compression.Algorithm compressAlgo; - - // Last key in the file. Filled in when we read in the file info - private byte [] lastkey = null; - // Stats read in when we load file info. - private int avgKeyLen = -1; - private int avgValueLen = -1; - - // Used to ensure we seek correctly. - RawComparator comparator; - - // Size of this file. - private final long fileSize; - - // Block cache to use. - private final BlockCache cache; - public int cacheHits = 0; - public int blockLoads = 0; - public int metaLoads = 0; - - // Whether file is from in-memory store - private boolean inMemory = false; - - // Whether blocks of file should be evicted on close of file - private final boolean evictOnClose; - - // Name for this object used when logging or in toString. Is either - // the result of a toString on the stream or else is toString of passed - // file Path plus metadata key/value pairs. - protected String name; - - /** - * Opens a HFile. You must load the file info before you can - * use it by calling {@link #loadFileInfo()}. - * - * @param fs filesystem to load from - * @param path path within said filesystem - * @param cache block cache. Pass null if none. - * @throws IOException - */ - public Reader(FileSystem fs, Path path, BlockCache cache, boolean inMemory, - boolean evictOnClose) - throws IOException { - this(path, fs.open(path), fs.getFileStatus(path).getLen(), cache, - inMemory, evictOnClose); - this.closeIStream = true; - this.name = path.toString(); - } - - /** - * Opens a HFile. You must load the index before you can - * use it by calling {@link #loadFileInfo()}. - * - * @param fsdis input stream. Caller is responsible for closing the passed - * stream. - * @param size Length of the stream. - * @param cache block cache. Pass null if none. - * @param inMemory whether blocks should be marked as in-memory in cache - * @param evictOnClose whether blocks in cache should be evicted on close - * @throws IOException - */ - public Reader(Path path, final FSDataInputStream fsdis, final long size, - final BlockCache cache, final boolean inMemory, - final boolean evictOnClose) { - this.cache = cache; - this.fileSize = size; - this.istream = fsdis; - this.closeIStream = false; - this.name = path.toString(); - this.inMemory = inMemory; - this.evictOnClose = evictOnClose; - } - - @Override - public String toString() { - return "reader=" + this.name + - (!isFileInfoLoaded()? "": - ", compression=" + this.compressAlgo.getName() + - ", inMemory=" + this.inMemory + - ", firstKey=" + toStringFirstKey() + - ", lastKey=" + toStringLastKey()) + - ", avgKeyLen=" + this.avgKeyLen + - ", avgValueLen=" + this.avgValueLen + - ", entries=" + this.trailer.entryCount + - ", length=" + this.fileSize; - } - - protected String toStringFirstKey() { - return KeyValue.keyToString(getFirstKey()); - } - - protected String toStringLastKey() { - return KeyValue.keyToString(getLastKey()); + public static final WriterFactory getWriterFactory(Configuration conf) { + int version = getFormatVersion(conf); + LOG.debug("Using HFile format version " + version); + switch (version) { + case 1: + return new HFileWriterV1.WriterFactoryV1(conf); + case 2: + return new HFileWriterV2.WriterFactoryV2(conf); + default: + throw new IllegalArgumentException("Cannot create writer for HFile " + + "format version " + version); } + } - public long length() { - return this.fileSize; - } + /** + * Configuration key to evict all blocks of a given file from the block cache + * when the file is closed. + */ + public static final String EVICT_BLOCKS_ON_CLOSE_KEY = + "hbase.rs.evictblocksonclose"; - public long getTotalUncompressedBytes() { - return this.trailer.totalUncompressedBytes; - } - - public boolean inMemory() { - return this.inMemory; - } + /** + * Configuration key to cache data blocks on write. There are separate + * switches for Bloom blocks and non-root index blocks. + */ + public static final String CACHE_BLOCKS_ON_WRITE_KEY = + "hbase.rs.cacheblocksonwrite"; - private byte[] readAllIndex(final FSDataInputStream in, final long indexOffset, - final int indexSize) throws IOException { - byte[] allIndex = new byte[indexSize]; - in.seek(indexOffset); - IOUtils.readFully(in, allIndex, 0, allIndex.length); - return allIndex; - } + /** An interface used by clients to open and iterate an {@link HFile}. */ + public interface Reader extends Closeable { /** - * Read in the index and file info. - * @return A map of fileinfo data. - * See {@link Writer#appendFileInfo(byte[], byte[])}. - * @throws IOException + * Returns this reader's "name". Usually the last component of the path. + * Needs to be constant as the file is being moved to support caching on + * write. */ - public Map loadFileInfo() - throws IOException { - this.trailer = readTrailer(); - - // Read in the fileinfo and get what we need from it. - this.istream.seek(this.trailer.fileinfoOffset); - FileInfo fi = new FileInfo(); - fi.readFields(this.istream); - this.lastkey = fi.get(FileInfo.LASTKEY); - this.avgKeyLen = Bytes.toInt(fi.get(FileInfo.AVG_KEY_LEN)); - this.avgValueLen = Bytes.toInt(fi.get(FileInfo.AVG_VALUE_LEN)); - String clazzName = Bytes.toString(fi.get(FileInfo.COMPARATOR)); - this.comparator = getComparator(clazzName); - - int allIndexSize = (int)(this.fileSize - this.trailer.dataIndexOffset - FixedFileTrailer.trailerSize()); - byte[] dataAndMetaIndex = readAllIndex(this.istream, this.trailer.dataIndexOffset, allIndexSize); - - ByteArrayInputStream bis = new ByteArrayInputStream(dataAndMetaIndex); - DataInputStream dis = new DataInputStream(bis); - - // Read in the data index. - this.blockIndex = - BlockIndex.readIndex(this.comparator, dis, this.trailer.dataIndexCount); - - // Read in the metadata index. - if (trailer.metaIndexCount > 0) { - this.metaIndex = BlockIndex.readIndex(Bytes.BYTES_RAWCOMPARATOR, dis, - this.trailer.metaIndexCount); - } - this.fileInfoLoaded = true; + String getName(); - if (null != dis) { - dis.close(); - } + String getColumnFamilyName(); - return fi; - } + RawComparator getComparator(); - boolean isFileInfoLoaded() { - return this.fileInfoLoaded; - } + HFileScanner getScanner(boolean cacheBlocks, + final boolean pread, final boolean isCompaction); - @SuppressWarnings("unchecked") - private RawComparator getComparator(final String clazzName) - throws IOException { - if (clazzName == null || clazzName.length() == 0) { - return null; - } - try { - return (RawComparator)Class.forName(clazzName).newInstance(); - } catch (InstantiationException e) { - throw new IOException(e); - } catch (IllegalAccessException e) { - throw new IOException(e); - } catch (ClassNotFoundException e) { - throw new IOException(e); - } - } + ByteBuffer getMetaBlock(String metaBlockName, + boolean cacheBlock) throws IOException; - /* Read the trailer off the input stream. As side effect, sets the - * compression algorithm. - * @return Populated FixedFileTrailer. - * @throws IOException - */ - private FixedFileTrailer readTrailer() throws IOException { - FixedFileTrailer fft = new FixedFileTrailer(); - long seekPoint = this.fileSize - FixedFileTrailer.trailerSize(); - this.istream.seek(seekPoint); - fft.deserialize(this.istream); - // Set up the codec. - this.compressAlgo = - Compression.Algorithm.values()[fft.compressionCodec]; - - CompressionTest.testCompression(this.compressAlgo); - - return fft; - } + HFileBlock readBlock(long offset, int onDiskBlockSize, + boolean cacheBlock, final boolean pread, final boolean isCompaction) + throws IOException; - /** - * Create a Scanner on this file. No seeks or reads are done on creation. - * Call {@link HFileScanner#seekTo(byte[])} to position an start the read. - * There is nothing to clean up in a Scanner. Letting go of your references - * to the scanner is sufficient. - * @param pread Use positional read rather than seek+read if true (pread is - * better for random reads, seek+read is better scanning). - * @param cacheBlocks True if we should cache blocks read in by this scanner. - * @return Scanner on this file. - */ - public HFileScanner getScanner(boolean cacheBlocks, final boolean pread) { - return new Scanner(this, cacheBlocks, pread); - } + Map loadFileInfo() throws IOException; - /** - * @param key Key to search. - * @return Block number of the block containing the key or -1 if not in this - * file. - */ - protected int blockContainingKey(final byte [] key, int offset, int length) { - if (blockIndex == null) { - throw new RuntimeException("Block index not loaded"); - } - return blockIndex.blockContainingKey(key, offset, length); - } - /** - * @param metaBlockName - * @param cacheBlock Add block to cache, if found - * @return Block wrapped in a ByteBuffer - * @throws IOException - */ - public ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock) - throws IOException { - if (trailer.metaIndexCount == 0) { - return null; // there are no meta blocks - } - if (metaIndex == null) { - throw new IOException("Meta index not loaded"); - } + byte[] getLastKey(); - byte [] mbname = Bytes.toBytes(metaBlockName); - int block = metaIndex.blockContainingKey(mbname, 0, mbname.length); - if (block == -1) - return null; - long blockSize; - if (block == metaIndex.count - 1) { - blockSize = trailer.fileinfoOffset - metaIndex.blockOffsets[block]; - } else { - blockSize = metaIndex.blockOffsets[block+1] - metaIndex.blockOffsets[block]; - } + byte[] midkey() throws IOException; - long now = System.currentTimeMillis(); - - // Per meta key from any given file, synchronize reads for said block - synchronized (metaIndex.blockKeys[block]) { - metaLoads++; - // Check cache for block. If found return. - if (cache != null) { - ByteBuffer cachedBuf = cache.getBlock(name + "meta" + block, - cacheBlock); - if (cachedBuf != null) { - // Return a distinct 'shallow copy' of the block, - // so pos doesnt get messed by the scanner - cacheHits++; - return cachedBuf.duplicate(); - } - // Cache Miss, please load. - } + long length(); - ByteBuffer buf = decompress(metaIndex.blockOffsets[block], - longToInt(blockSize), metaIndex.blockDataSizes[block], true); - byte [] magic = new byte[METABLOCKMAGIC.length]; - buf.get(magic, 0, magic.length); + long getEntries(); - if (! Arrays.equals(magic, METABLOCKMAGIC)) { - throw new IOException("Meta magic is bad in block " + block); - } + byte[] getFirstKey(); - // Create a new ByteBuffer 'shallow copy' to hide the magic header - buf = buf.slice(); + long indexSize(); - readTime += System.currentTimeMillis() - now; - readOps++; + byte[] getFirstRowKey(); - // Cache the block - if(cacheBlock && cache != null) { - cache.cacheBlock(name + "meta" + block, buf.duplicate(), inMemory); - } + byte[] getLastRowKey(); - return buf; - } - } + FixedFileTrailer getTrailer(); - /** - * Read in a file block. - * @param block Index of block to read. - * @param pread Use positional read instead of seek+read (positional is - * better doing random reads whereas seek+read is better scanning). - * @return Block wrapped in a ByteBuffer. - * @throws IOException - */ - ByteBuffer readBlock(int block, boolean cacheBlock, final boolean pread) - throws IOException { - if (blockIndex == null) { - throw new IOException("Block index not loaded"); - } - if (block < 0 || block >= blockIndex.count) { - throw new IOException("Requested block is out of range: " + block + - ", max: " + blockIndex.count); - } - // For any given block from any given file, synchronize reads for said - // block. - // Without a cache, this synchronizing is needless overhead, but really - // the other choice is to duplicate work (which the cache would prevent you from doing). - synchronized (blockIndex.blockKeys[block]) { - blockLoads++; - // Check cache for block. If found return. - if (cache != null) { - ByteBuffer cachedBuf = cache.getBlock(name + block, cacheBlock); - if (cachedBuf != null) { - // Return a distinct 'shallow copy' of the block, - // so pos doesnt get messed by the scanner - cacheHits++; - return cachedBuf.duplicate(); - } - // Carry on, please load. - } - - // Load block from filesystem. - long now = System.currentTimeMillis(); - long onDiskBlockSize; - if (block == blockIndex.count - 1) { - // last block! The end of data block is first meta block if there is - // one or if there isn't, the fileinfo offset. - long offset = this.metaIndex != null? - this.metaIndex.blockOffsets[0]: this.trailer.fileinfoOffset; - onDiskBlockSize = offset - blockIndex.blockOffsets[block]; - } else { - onDiskBlockSize = blockIndex.blockOffsets[block+1] - - blockIndex.blockOffsets[block]; - } - ByteBuffer buf = decompress(blockIndex.blockOffsets[block], - longToInt(onDiskBlockSize), this.blockIndex.blockDataSizes[block], - pread); - - byte [] magic = new byte[DATABLOCKMAGIC.length]; - buf.get(magic, 0, magic.length); - if (!Arrays.equals(magic, DATABLOCKMAGIC)) { - throw new IOException("Data magic is bad in block " + block); - } - - // 'shallow copy' to hide the header - // NOTE: you WILL GET BIT if you call buf.array() but don't start - // reading at buf.arrayOffset() - buf = buf.slice(); - - readTime += System.currentTimeMillis() - now; - readOps++; - - // Cache the block - if(cacheBlock && cache != null) { - cache.cacheBlock(name + block, buf.duplicate(), inMemory); - } - - return buf; - } - } + HFileBlockIndex.BlockIndexReader getDataBlockIndexReader(); - /* - * Decompress compressedSize bytes off the backing - * FSDataInputStream. - * @param offset - * @param compressedSize - * @param decompressedSize - * - * @return - * @throws IOException - */ - private ByteBuffer decompress(final long offset, final int compressedSize, - final int decompressedSize, final boolean pread) - throws IOException { - Decompressor decompressor = null; - ByteBuffer buf = null; - try { - decompressor = this.compressAlgo.getDecompressor(); - // My guess is that the bounded range fis is needed to stop the - // decompressor reading into next block -- IIRC, it just grabs a - // bunch of data w/o regard to whether decompressor is coming to end of a - // decompression. - - // We use a buffer of DEFAULT_BLOCKSIZE size. This might be extreme. - // Could maybe do with less. Study and figure it: TODO - InputStream is = this.compressAlgo.createDecompressionStream( - new BufferedInputStream( - new BoundedRangeFileInputStream(this.istream, offset, compressedSize, - pread), - Math.min(DEFAULT_BLOCKSIZE, compressedSize)), - decompressor, 0); - buf = ByteBuffer.allocate(decompressedSize); - IOUtils.readFully(is, buf.array(), 0, buf.capacity()); - is.close(); - } finally { - if (null != decompressor) { - this.compressAlgo.returnDecompressor(decompressor); - } - } - return buf; - } + HFileScanner getScanner(boolean cacheBlocks, boolean pread); - /** - * @return First key in the file. May be null if file has no entries. - * Note that this is not the first rowkey, but rather the byte form of - * the first KeyValue. - */ - public byte [] getFirstKey() { - if (blockIndex == null) { - throw new RuntimeException("Block index not loaded"); - } - return this.blockIndex.isEmpty()? null: this.blockIndex.blockKeys[0]; - } + Compression.Algorithm getCompressionAlgorithm(); /** - * @return the first row key, or null if the file is empty. - * TODO move this to StoreFile after Ryan's patch goes in - * to eliminate KeyValue here + * Retrieves Bloom filter metadata as appropriate for each {@link HFile} + * version. Knows nothing about how that metadata is structured. */ - public byte[] getFirstRowKey() { - byte[] firstKey = getFirstKey(); - if (firstKey == null) return null; - return KeyValue.createKeyValueFromKey(firstKey).getRow(); - } - - /** - * @return number of KV entries in this HFile - */ - public int getEntries() { - if (!this.isFileInfoLoaded()) { - throw new RuntimeException("File info not loaded"); - } - return this.trailer.entryCount; - } - - /** - * @return Last key in the file. May be null if file has no entries. - * Note that this is not the last rowkey, but rather the byte form of - * the last KeyValue. - */ - public byte [] getLastKey() { - if (!isFileInfoLoaded()) { - throw new RuntimeException("Load file info first"); - } - return this.blockIndex.isEmpty()? null: this.lastkey; - } - - /** - * @return the last row key, or null if the file is empty. - * TODO move this to StoreFile after Ryan's patch goes in - * to eliminate KeyValue here - */ - public byte[] getLastRowKey() { - byte[] lastKey = getLastKey(); - if (lastKey == null) return null; - return KeyValue.createKeyValueFromKey(lastKey).getRow(); - } - - /** - * @return number of K entries in this HFile's filter. Returns KV count if no filter. - */ - public int getFilterEntries() { - return getEntries(); - } - - /** - * @return Comparator. - */ - public RawComparator getComparator() { - return this.comparator; - } - - public Compression.Algorithm getCompressionAlgorithm() { - return this.compressAlgo; - } - - /** - * @return index size - */ - public long indexSize() { - return (this.blockIndex != null? this.blockIndex.heapSize(): 0) + - ((this.metaIndex != null)? this.metaIndex.heapSize(): 0); - } - - /** - * @return Midkey for this file. We work with block boundaries only so - * returned midkey is an approximation only. - * @throws IOException - */ - public byte [] midkey() throws IOException { - if (!isFileInfoLoaded() || this.blockIndex.isEmpty()) { - return null; - } - return this.blockIndex.midkey(); - } - - public void close() throws IOException { - if (evictOnClose && this.cache != null) { - int numEvicted = 0; - for (int i=0; i= reader.blockIndex.count) { - // damn we are at the end - currBlock = 0; - block = null; - return false; - } - block = reader.readBlock(this.currBlock, this.cacheBlocks, this.pread); - currKeyLen = Bytes.toInt(block.array(), block.arrayOffset()+block.position(), 4); - currValueLen = Bytes.toInt(block.array(), block.arrayOffset()+block.position()+4, 4); - block.position(block.position()+8); - blockFetches++; - return true; - } - // LOG.debug("rem:" + block.remaining() + " p:" + block.position() + - // " kl: " + currKeyLen + " kv: " + currValueLen); - currKeyLen = Bytes.toInt(block.array(), block.arrayOffset()+block.position(), 4); - currValueLen = Bytes.toInt(block.array(), block.arrayOffset()+block.position()+4, 4); - block.position(block.position()+8); - return true; - } - - public int seekTo(byte [] key) throws IOException { - return seekTo(key, 0, key.length); - } - - public int seekTo(byte[] key, int offset, int length) throws IOException { - int b = reader.blockContainingKey(key, offset, length); - if (b < 0) return -1; // falls before the beginning of the file! :-( - // Avoid re-reading the same block (that'd be dumb). - loadBlock(b, true); - return blockSeek(key, offset, length, false); - } - - public int reseekTo(byte [] key) throws IOException { - return reseekTo(key, 0, key.length); - } - - public int reseekTo(byte[] key, int offset, int length) - throws IOException { - - if (this.block != null && this.currKeyLen != 0) { - ByteBuffer bb = getKey(); - int compared = this.reader.comparator.compare(key, offset, length, - bb.array(), bb.arrayOffset(), bb.limit()); - if (compared < 1) { - //If the required key is less than or equal to current key, then - //don't do anything. - return compared; - } - } - - int b = reader.blockContainingKey(key, offset, length); - if (b < 0) { - return -1; - } - loadBlock(b, false); - return blockSeek(key, offset, length, false); - } - - /** - * Within a loaded block, seek looking for the first key - * that is smaller than (or equal to?) the key we are interested in. - * - * A note on the seekBefore - if you have seekBefore = true, AND the - * first key in the block = key, then you'll get thrown exceptions. - * @param key to find - * @param seekBefore find the key before the exact match. - * @return - */ - private int blockSeek(byte[] key, int offset, int length, boolean seekBefore) { - int klen, vlen; - int lastLen = 0; - do { - klen = block.getInt(); - vlen = block.getInt(); - int comp = this.reader.comparator.compare(key, offset, length, - block.array(), block.arrayOffset() + block.position(), klen); - if (comp == 0) { - if (seekBefore) { - block.position(block.position() - lastLen - 16); - currKeyLen = block.getInt(); - currValueLen = block.getInt(); - return 1; // non exact match. - } - currKeyLen = klen; - currValueLen = vlen; - return 0; // indicate exact match - } - if (comp < 0) { - // go back one key: - block.position(block.position() - lastLen - 16); - currKeyLen = block.getInt(); - currValueLen = block.getInt(); - return 1; - } - block.position(block.position() + klen + vlen); - lastLen = klen + vlen ; - } while(block.remaining() > 0); - // ok we are at the end, so go back a littleeeeee.... - // The 8 in the below is intentionally different to the 16s in the above - // Do the math you you'll figure it. - block.position(block.position() - lastLen - 8); - currKeyLen = block.getInt(); - currValueLen = block.getInt(); - return 1; // didn't exactly find it. - } - - public boolean seekBefore(byte [] key) throws IOException { - return seekBefore(key, 0, key.length); - } - - public boolean seekBefore(byte[] key, int offset, int length) - throws IOException { - int b = reader.blockContainingKey(key, offset, length); - if (b < 0) - return false; // key is before the start of the file. - - // Question: does this block begin with 'key'? - if (this.reader.comparator.compare(reader.blockIndex.blockKeys[b], - 0, reader.blockIndex.blockKeys[b].length, - key, offset, length) == 0) { - // Ok the key we're interested in is the first of the block, so go back one. - if (b == 0) { - // we have a 'problem', the key we want is the first of the file. - return false; - } - b--; - // TODO shortcut: seek forward in this block to the last key of the block. - } - loadBlock(b, true); - blockSeek(key, offset, length, true); - return true; - } - - public String getKeyString() { - return Bytes.toStringBinary(block.array(), block.arrayOffset() + - block.position(), currKeyLen); - } - - public String getValueString() { - return Bytes.toString(block.array(), block.arrayOffset() + - block.position() + currKeyLen, currValueLen); - } - - public Reader getReader() { - return this.reader; - } - - public boolean isSeeked(){ - return this.block != null; - } - - public boolean seekTo() throws IOException { - if (this.reader.blockIndex.isEmpty()) { - return false; - } - if (block != null && currBlock == 0) { - block.rewind(); - currKeyLen = block.getInt(); - currValueLen = block.getInt(); - return true; - } - currBlock = 0; - block = reader.readBlock(this.currBlock, this.cacheBlocks, this.pread); - currKeyLen = block.getInt(); - currValueLen = block.getInt(); - blockFetches++; - return true; - } - - private void loadBlock(int bloc, boolean rewind) throws IOException { - if (block == null) { - block = reader.readBlock(bloc, this.cacheBlocks, this.pread); - currBlock = bloc; - blockFetches++; - } else { - if (bloc != currBlock) { - block = reader.readBlock(bloc, this.cacheBlocks, this.pread); - currBlock = bloc; - blockFetches++; - } else { - // we are already in the same block, just rewind to seek again. - if (rewind) { - block.rewind(); - } - else { - //Go back by (size of rowlength + size of valuelength) = 8 bytes - block.position(block.position()-8); - } - } - } - } - - @Override - public String toString() { - return "HFileScanner for reader " + String.valueOf(reader); - } - } - - public String getTrailerInfo() { - return trailer.toString(); - } + DataInput getBloomFilterMetadata() throws IOException; } - /* - * The RFile has a fixed trailer which contains offsets to other variable - * parts of the file. Also includes basic metadata on this file. - */ - private static class FixedFileTrailer { - // Offset to the fileinfo data, a small block of vitals.. - long fileinfoOffset; - // Offset to the data block index. - long dataIndexOffset; - // How many index counts are there (aka: block count) - int dataIndexCount; - // Offset to the meta block index. - long metaIndexOffset; - // How many meta block index entries (aka: meta block count) - int metaIndexCount; - long totalUncompressedBytes; - int entryCount; - int compressionCodec; - int version = 1; - - FixedFileTrailer() { - super(); - } - - static int trailerSize() { - // Keep this up to date... - return - ( Bytes.SIZEOF_INT * 5 ) + - ( Bytes.SIZEOF_LONG * 4 ) + - TRAILERBLOCKMAGIC.length; - } - - void serialize(DataOutputStream outputStream) throws IOException { - outputStream.write(TRAILERBLOCKMAGIC); - outputStream.writeLong(fileinfoOffset); - outputStream.writeLong(dataIndexOffset); - outputStream.writeInt(dataIndexCount); - outputStream.writeLong(metaIndexOffset); - outputStream.writeInt(metaIndexCount); - outputStream.writeLong(totalUncompressedBytes); - outputStream.writeInt(entryCount); - outputStream.writeInt(compressionCodec); - outputStream.writeInt(version); - } - - void deserialize(DataInputStream inputStream) throws IOException { - byte [] header = new byte[TRAILERBLOCKMAGIC.length]; - inputStream.readFully(header); - if ( !Arrays.equals(header, TRAILERBLOCKMAGIC)) { - throw new IOException("Trailer 'header' is wrong; does the trailer " + - "size match content?"); - } - fileinfoOffset = inputStream.readLong(); - dataIndexOffset = inputStream.readLong(); - dataIndexCount = inputStream.readInt(); - - metaIndexOffset = inputStream.readLong(); - metaIndexCount = inputStream.readInt(); - - totalUncompressedBytes = inputStream.readLong(); - entryCount = inputStream.readInt(); - compressionCodec = inputStream.readInt(); - version = inputStream.readInt(); - - if (version != 1) { - throw new IOException("Wrong version: " + version); - } - } - - @Override - public String toString() { - return "fileinfoOffset=" + fileinfoOffset + - ", dataIndexOffset=" + dataIndexOffset + - ", dataIndexCount=" + dataIndexCount + - ", metaIndexOffset=" + metaIndexOffset + - ", metaIndexCount=" + metaIndexCount + - ", totalBytes=" + totalUncompressedBytes + - ", entryCount=" + entryCount + - ", version=" + version; + private static Reader pickReaderVersion(Path path, FSDataInputStream fsdis, + long size, boolean closeIStream, BlockCache blockCache, + boolean inMemory, boolean evictOnClose) throws IOException { + FixedFileTrailer trailer = FixedFileTrailer.readFromStream(fsdis, size); + switch (trailer.getVersion()) { + case 1: + return new HFileReaderV1(path, trailer, fsdis, size, closeIStream, + blockCache, inMemory, evictOnClose); + case 2: + return new HFileReaderV2(path, trailer, fsdis, size, closeIStream, + blockCache, inMemory, evictOnClose); + default: + throw new IOException("Cannot instantiate reader for HFile version " + + trailer.getVersion()); } } - /* - * The block index for a RFile. - * Used reading. - */ - static class BlockIndex implements HeapSize { - // How many actual items are there? The next insert location too. - int count = 0; - byte [][] blockKeys; - long [] blockOffsets; - int [] blockDataSizes; - int size = 0; - - /* Needed doing lookup on blocks. - */ - final RawComparator comparator; - - /* - * Shutdown default constructor - */ - @SuppressWarnings("unused") - private BlockIndex() { - this(null); - } - - - /** - * @param c comparator used to compare keys. - */ - BlockIndex(final RawComparatorc) { - this.comparator = c; - // Guess that cost of three arrays + this object is 4 * 8 bytes. - this.size += (4 * 8); - } - - /** - * @return True if block index is empty. - */ - boolean isEmpty() { - return this.blockKeys.length <= 0; - } - - /** - * Adds a new entry in the block index. - * - * @param key Last key in the block - * @param offset file offset where the block is stored - * @param dataSize the uncompressed data size - */ - void add(final byte[] key, final long offset, final int dataSize) { - blockOffsets[count] = offset; - blockKeys[count] = key; - blockDataSizes[count] = dataSize; - count++; - this.size += (Bytes.SIZEOF_INT * 2 + key.length); - } - - /** - * @param key Key to find - * @return Offset of block containing key or -1 if this file - * does not contain the request. - */ - int blockContainingKey(final byte[] key, int offset, int length) { - int pos = Bytes.binarySearch(blockKeys, key, offset, length, this.comparator); - if (pos < 0) { - pos ++; - pos *= -1; - if (pos == 0) { - // falls before the beginning of the file. - return -1; - } - // When switched to "first key in block" index, binarySearch now returns - // the block with a firstKey < key. This means the value we want is potentially - // in the next block. - pos --; // in previous block. - - return pos; - } - // wow, a perfect hit, how unlikely? - return pos; - } - - /* - * @return File midkey. Inexact. Operates on block boundaries. Does - * not go into blocks. - */ - byte [] midkey() throws IOException { - int pos = ((this.count - 1)/2); // middle of the index - if (pos < 0) { - throw new IOException("HFile empty"); - } - return this.blockKeys[pos]; - } - - /* - * Write out index. Whatever we write here must jibe with what - * BlockIndex#readIndex is expecting. Make sure the two ends of the - * index serialization match. - * @param o - * @param keys - * @param offsets - * @param sizes - * @param c - * @return Position at which we entered the index. - * @throws IOException - */ - static long writeIndex(final FSDataOutputStream o, - final List keys, final List offsets, - final List sizes) - throws IOException { - long pos = o.getPos(); - // Don't write an index if nothing in the index. - if (keys.size() > 0) { - o.write(INDEXBLOCKMAGIC); - // Write the index. - for (int i = 0; i < keys.size(); ++i) { - o.writeLong(offsets.get(i).longValue()); - o.writeInt(sizes.get(i).intValue()); - byte [] key = keys.get(i); - Bytes.writeByteArray(o, key); - } - } - return pos; - } - - /* - * Read in the index that is at indexOffset - * Must match what was written by writeIndex in the Writer.close. - * @param c Comparator to use. - * @param in - * @param indexSize - * @throws IOException - */ - static BlockIndex readIndex(final RawComparator c, - DataInputStream in, final int indexSize) - throws IOException { - BlockIndex bi = new BlockIndex(c); - bi.blockOffsets = new long[indexSize]; - bi.blockKeys = new byte[indexSize][]; - bi.blockDataSizes = new int[indexSize]; - // If index size is zero, no index was written. - if (indexSize > 0) { - byte [] magic = new byte[INDEXBLOCKMAGIC.length]; - in.readFully(magic); - if (!Arrays.equals(magic, INDEXBLOCKMAGIC)) { - throw new IOException("Index block magic is wrong: " + - Arrays.toString(magic)); - } - for (int i = 0; i < indexSize; ++i ) { - long offset = in.readLong(); - int dataSize = in.readInt(); - byte [] key = Bytes.readByteArray(in); - bi.add(key, offset, dataSize); - } - } - return bi; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("size=" + count).append("\n"); - for (int i = 0; i < count ; i++) { - sb.append("key=").append(KeyValue.keyToString(blockKeys[i])). - append("\n offset=").append(blockOffsets[i]). - append(", dataSize=" + blockDataSizes[i]). - append("\n"); - } - return sb.toString(); - } - - public long heapSize() { - long heapsize = ClassSize.align(ClassSize.OBJECT + - 2 * Bytes.SIZEOF_INT + (3 + 1) * ClassSize.REFERENCE); - //Calculating the size of blockKeys - if(blockKeys != null) { - //Adding array + references overhead - heapsize += ClassSize.align(ClassSize.ARRAY + - blockKeys.length * ClassSize.REFERENCE); - //Adding bytes - for(byte [] bs : blockKeys) { - heapsize += ClassSize.align(ClassSize.ARRAY + bs.length); - } - } - if(blockOffsets != null) { - heapsize += ClassSize.align(ClassSize.ARRAY + - blockOffsets.length * Bytes.SIZEOF_LONG); - } - if(blockDataSizes != null) { - heapsize += ClassSize.align(ClassSize.ARRAY + - blockDataSizes.length * Bytes.SIZEOF_INT); - } - - return ClassSize.align(heapsize); - } + public static Reader createReader( + FileSystem fs, Path path, BlockCache blockCache, boolean inMemory, + boolean evictOnClose) throws IOException { + return pickReaderVersion(path, fs.open(path), + fs.getFileStatus(path).getLen(), true, blockCache, inMemory, + evictOnClose); + } + public static Reader createReader(Path path, FSDataInputStream fsdis, + long size, BlockCache blockache, boolean inMemory, boolean evictOnClose) + throws IOException { + return pickReaderVersion(path, fsdis, size, false, blockache, inMemory, + evictOnClose); } /* @@ -1825,23 +366,37 @@ public class HFile { static final byte [] COMPARATOR = Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR"); - /* - * Constructor. - */ - FileInfo() { - super(); + /** + * Append the given key/value pair to the file info, optionally checking the + * key prefix. + * + * @param k key to add + * @param v value to add + * @param checkPrefix whether to check that the provided key does not start + * with the reserved prefix + * @return this file info object + * @throws IOException if the key or value is invalid + */ + public FileInfo append(final byte[] k, final byte[] v, + final boolean checkPrefix) throws IOException { + if (k == null || v == null) { + throw new NullPointerException("Key nor value may be null"); + } + if (checkPrefix && isReservedFileInfoKey(k)) { + throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX + + " are reserved"); + } + put(k, v); + return this; } + } - /** - * Return true if the given file info key is reserved for internal - * use by HFile. - */ + /** Return true if the given file info key is reserved for internal use. */ public static boolean isReservedFileInfoKey(byte[] key) { return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES); } - /** * Get names of supported compression algorithms. The names are acceptable by * HFile.Writer. @@ -1879,7 +434,7 @@ public class HFile { * @throws IOException When scanning the files fails. */ static List getStoreFiles(FileSystem fs, Path regionDir) - throws IOException { + throws IOException { List res = new ArrayList(); PathFilter dirFilter = new FSUtils.DirFilter(fs); FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter); @@ -1894,171 +449,31 @@ public class HFile { return res; } - public static void main(String []args) throws IOException { - try { - // create options - Options options = new Options(); - options.addOption("v", "verbose", false, "Verbose output; emits file and meta data delimiters"); - options.addOption("p", "printkv", false, "Print key/value pairs"); - options.addOption("e", "printkey", false, "Print keys"); - options.addOption("m", "printmeta", false, "Print meta data of file"); - options.addOption("b", "printblocks", false, "Print block index meta data"); - options.addOption("k", "checkrow", false, - "Enable row order check; looks for out-of-order keys"); - options.addOption("a", "checkfamily", false, "Enable family check"); - options.addOption("f", "file", true, - "File to scan. Pass full-path; e.g. hdfs://a:9000/hbase/.META./12/34"); - options.addOption("r", "region", true, - "Region to scan. Pass region name; e.g. '.META.,,1'"); - if (args.length == 0) { - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp("HFile ", options, true); - System.exit(-1); - } - CommandLineParser parser = new PosixParser(); - CommandLine cmd = parser.parse(options, args); - boolean verbose = cmd.hasOption("v"); - boolean printValue = cmd.hasOption("p"); - boolean printKey = cmd.hasOption("e") || printValue; - boolean printMeta = cmd.hasOption("m"); - boolean printBlocks = cmd.hasOption("b"); - boolean checkRow = cmd.hasOption("k"); - boolean checkFamily = cmd.hasOption("a"); - // get configuration, file system and get list of files - Configuration conf = HBaseConfiguration.create(); - conf.set("fs.defaultFS", - conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR)); - conf.set("fs.default.name", - conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR)); - ArrayList files = new ArrayList(); - if (cmd.hasOption("f")) { - files.add(new Path(cmd.getOptionValue("f"))); - } - if (cmd.hasOption("r")) { - String regionName = cmd.getOptionValue("r"); - byte[] rn = Bytes.toBytes(regionName); - byte[][] hri = HRegionInfo.parseRegionName(rn); - Path rootDir = FSUtils.getRootDir(conf); - Path tableDir = new Path(rootDir, Bytes.toString(hri[0])); - String enc = HRegionInfo.encodeRegionName(rn); - Path regionDir = new Path(tableDir, enc); - if (verbose) System.out.println("region dir -> " + regionDir); - List regionFiles = - getStoreFiles(FileSystem.get(conf), regionDir); - if (verbose) System.out.println("Number of region files found -> " + - regionFiles.size()); - if (verbose) { - int i = 1; - for (Path p : regionFiles) { - if (verbose) System.out.println("Found file[" + i++ + "] -> " + p); - } - } - files.addAll(regionFiles); - } - // iterate over all files found - for (Path file : files) { - if (verbose) System.out.println("Scanning -> " + file); - FileSystem fs = file.getFileSystem(conf); - if (!fs.exists(file)) { - System.err.println("ERROR, file doesnt exist: " + file); - continue; - } - // create reader and load file info - HFile.Reader reader = new HFile.Reader(fs, file, null, false, false); - Map fileInfo = reader.loadFileInfo(); - int count = 0; - if (verbose || printKey || checkRow || checkFamily) { - // scan over file and read key/value's and check if requested - HFileScanner scanner = reader.getScanner(false, false); - scanner.seekTo(); - KeyValue pkv = null; - do { - KeyValue kv = scanner.getKeyValue(); - // dump key value - if (printKey) { - System.out.print("K: " + kv); - if (printValue) { - System.out.print(" V: " + Bytes.toStringBinary(kv.getValue())); - } - System.out.println(); - } - // check if rows are in order - if (checkRow && pkv != null) { - if (Bytes.compareTo(pkv.getRow(), kv.getRow()) > 0) { - System.err.println("WARNING, previous row is greater then" + - " current row\n\tfilename -> " + file + - "\n\tprevious -> " + Bytes.toStringBinary(pkv.getKey()) + - "\n\tcurrent -> " + Bytes.toStringBinary(kv.getKey())); - } - } - // check if families are consistent - if (checkFamily) { - String fam = Bytes.toString(kv.getFamily()); - if (!file.toString().contains(fam)) { - System.err.println("WARNING, filename does not match kv family," + - "\n\tfilename -> " + file + - "\n\tkeyvalue -> " + Bytes.toStringBinary(kv.getKey())); - } - if (pkv != null && !Bytes.equals(pkv.getFamily(), kv.getFamily())) { - System.err.println("WARNING, previous kv has different family" + - " compared to current key\n\tfilename -> " + file + - "\n\tprevious -> " + Bytes.toStringBinary(pkv.getKey()) + - "\n\tcurrent -> " + Bytes.toStringBinary(kv.getKey())); - } - } - pkv = kv; - count++; - } while (scanner.next()); - } - if (verbose || printKey) { - System.out.println("Scanned kv count -> " + count); - } - // print meta data - if (printMeta) { - System.out.println("Block index size as per heapsize: " + reader.indexSize()); - System.out.println(reader.toString()); - System.out.println(reader.getTrailerInfo()); - System.out.println("Fileinfo:"); - for (Map.Entry e : fileInfo.entrySet()) { - System.out.print(Bytes.toString(e.getKey()) + " = " ); - if (Bytes.equals(e.getKey(), Bytes.toBytes("MAX_SEQ_ID_KEY"))) { - long seqid = Bytes.toLong(e.getValue()); - System.out.println(seqid); - } else if (Bytes.equals(e.getKey(), - Bytes.toBytes("TIMERANGE"))) { - TimeRangeTracker timeRangeTracker = new TimeRangeTracker(); - Writables.copyWritable(e.getValue(), timeRangeTracker); - System.out.println(timeRangeTracker.getMinimumTimestamp() + - "...." + timeRangeTracker.getMaximumTimestamp()); - } else if (Bytes.equals(e.getKey(), FileInfo.AVG_KEY_LEN) || - Bytes.equals(e.getKey(), FileInfo.AVG_VALUE_LEN)) { - System.out.println(Bytes.toInt(e.getValue())); - } else { - System.out.println(Bytes.toStringBinary(e.getValue())); - } - } - - //Printing bloom information - ByteBuffer b = reader.getMetaBlock("BLOOM_FILTER_META", false); - if (b!= null) { - BloomFilter bloomFilter = new ByteBloomFilter(b); - System.out.println("BloomSize: " + bloomFilter.getByteSize()); - System.out.println("No of Keys in bloom: " + - bloomFilter.getKeyCount()); - System.out.println("Max Keys for bloom: " + - bloomFilter.getMaxKeys()); - } else { - System.out.println("Could not get bloom data from meta block"); - } - } - if (printBlocks) { - System.out.println("Block Index:"); - System.out.println(reader.blockIndex); - } - reader.close(); - } - } catch (Exception e) { - e.printStackTrace(); + public static void main(String[] args) throws IOException { + HFilePrettyPrinter prettyPrinter = new HFilePrettyPrinter(); + System.exit(prettyPrinter.run(args)); + } + + public static String getBlockCacheKey(String hfileName, long offset) { + return hfileName + CACHE_KEY_SEPARATOR + offset; + } + + /** + * Checks the given {@link HFile} format version, and throws an exception if + * invalid. Note that if the version number comes from an input file and has + * not been verified, the caller needs to re-throw an {@link IOException} to + * indicate that this is not a software error, but corrupted input. + * + * @param version an HFile version + * @throws IllegalArgumentException if the version is invalid + */ + public static void checkFormatVersion(int version) + throws IllegalArgumentException { + if (version < MIN_FORMAT_VERSION || version > MAX_FORMAT_VERSION) { + throw new IllegalArgumentException("Invalid HFile version: " + version + + " (expected to be " + "between " + MIN_FORMAT_VERSION + " and " + + MAX_FORMAT_VERSION + ")"); } } + } diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java new file mode 100644 index 0000000..097dc50 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java @@ -0,0 +1,1441 @@ +/* + * Copyright 2011 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.BufferedInputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; + +import org.apache.hadoop.hbase.io.DoubleOutputStream; +import org.apache.hadoop.hbase.io.HeapSize; +import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.hbase.util.CompoundBloomFilter; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.Decompressor; + +import com.google.common.base.Preconditions; + +import static org.apache.hadoop.hbase.io.hfile.BlockType.MAGIC_LENGTH; +import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.NONE; + +/** + * Reading {@link HFile} version 1 and 2 blocks, and writing version 2 blocks. + *
    + *
  • In version 1 all blocks are always compressed or uncompressed, as + * specified by the {@link HFile}'s compression algorithm, with a type-specific + * magic record stored in the beginning of the compressed data (i.e. one needs + * to uncompress the compressed block to determine the block type). There is + * only a single compression algorithm setting for all blocks. Offset and size + * information from the block index are required to read a block. + *
  • In version 2 a block is structured as follows: + *
      + *
    • Magic record identifying the block type (8 bytes) + *
    • Compressed block size, header not included (4 bytes) + *
    • Uncompressed block size, header not included (4 bytes) + *
    • The offset of the previous block of the same type (8 bytes). This is + * used to be able to navigate to the previous block without going to the block + * index. + *
    • Compressed data (or uncompressed data if compression is disabled). The + * compression algorithm is the same for all the blocks in the {@link HFile}, + * similarly to what was done in version 1. + *
    + *
+ * The version 2 block representation in the block cache is the same as above, + * except that the data section is always uncompressed in the cache. + */ +public class HFileBlock implements HeapSize { + + /** The size of a version 2 {@link HFile} block header */ + public static final int HEADER_SIZE = MAGIC_LENGTH + 2 * Bytes.SIZEOF_INT + + Bytes.SIZEOF_LONG; + + /** Just an array of bytes of the right size. */ + public static final byte[] DUMMY_HEADER = new byte[HEADER_SIZE]; + + public static final int BYTE_BUFFER_HEAP_SIZE = (int) ClassSize.estimateBase( + ByteBuffer.wrap(new byte[0], 0, 0).getClass(), false); + + private BlockType blockType; + private final int onDiskSizeWithoutHeader; + private final int uncompressedSizeWithoutHeader; + private final long prevBlockOffset; + private ByteBuffer buf; + + /** + * The offset of this block in the file. Populated by the reader for + * convenience of access. This offset is not part of the block header. + */ + private long offset = -1; + + /** + * The on-disk size of the next block, including the header, obtained by + * peeking into the first {@link HEADER_SIZE} bytes of the next block's + * header, or -1 if unknown. + */ + private int nextBlockOnDiskSizeWithHeader = -1; + + /** + * Creates a new {@link HFile} block from the given fields. This constructor + * is mostly used when the block data has already been read and uncompressed, + * and is sitting in a byte buffer. + * + * @param blockType the type of this block, see {@link BlockType} + * @param onDiskSizeWithoutHeader compressed size of the block if compression + * is used, otherwise uncompressed size, header size not included + * @param uncompressedSizeWithoutHeader uncompressed size of the block, + * header size not included. Equals onDiskSizeWithoutHeader if + * compression is disabled. + * @param prevBlockOffset the offset of the previous block in the + * {@link HFile} + * @param buf block header ({@link #HEADER_SIZE} bytes) followed by + * uncompressed data. This + * @param fillHeader true to fill in the first {@link #HEADER_SIZE} bytes of + * the buffer based on the header fields provided + * @param offset the file offset the block was read from + */ + public HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, + int uncompressedSizeWithoutHeader, long prevBlockOffset, ByteBuffer buf, + boolean fillHeader, long offset) { + this.blockType = blockType; + this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader; + this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader; + this.prevBlockOffset = prevBlockOffset; + this.buf = buf; + if (fillHeader) + overwriteHeader(); + this.offset = offset; + } + + /** + * Creates a block from an existing buffer starting with a header. Rewinds + * and takes ownership of the buffer. By definition of rewind, ignores the + * buffer position, but if you slice the buffer beforehand, it will rewind + * to that point. + */ + private HFileBlock(ByteBuffer b) throws IOException { + b.rewind(); + blockType = BlockType.read(b); + onDiskSizeWithoutHeader = b.getInt(); + uncompressedSizeWithoutHeader = b.getInt(); + prevBlockOffset = b.getLong(); + buf = b; + buf.rewind(); + } + + public BlockType getBlockType() { + return blockType; + } + + /** + * @return the on-disk size of the block with header size included + */ + public int getOnDiskSizeWithHeader() { + return onDiskSizeWithoutHeader + HEADER_SIZE; + } + + /** + * Returns the size of the compressed part of the block in case compression + * is used, or the uncompressed size of the data part otherwise. Header size + * is not included. + * + * @return the on-disk size of the data part of the block, header not + * included + */ + public int getOnDiskSizeWithoutHeader() { + return onDiskSizeWithoutHeader; + } + + /** + * @return the uncompressed size of the data part of the block, header not + * included + */ + public int getUncompressedSizeWithoutHeader() { + return uncompressedSizeWithoutHeader; + } + + /** + * @return the offset of the previous block of the same type in the file, or + * -1 if unknown + */ + public long getPrevBlockOffset() { + return prevBlockOffset; + } + + /** + * Writes header fields into the first {@link HEADER_SIZE} bytes of the + * buffer. Resets the buffer position to the end of header as side effect. + */ + private void overwriteHeader() { + buf.rewind(); + blockType.write(buf); + buf.putInt(onDiskSizeWithoutHeader); + buf.putInt(uncompressedSizeWithoutHeader); + buf.putLong(prevBlockOffset); + } + + /** + * Returns a buffer that does not include the header. The array offset points + * to the start of the block data right after the header. The underlying data + * array is not copied. + * + * @return the buffer with header skipped + */ + public ByteBuffer getBufferWithoutHeader() { + return ByteBuffer.wrap(buf.array(), buf.arrayOffset() + HEADER_SIZE, + buf.limit() - HEADER_SIZE).slice(); + } + + /** + * Returns the buffer this block stores internally. The clients must not + * modify the buffer object. This method has to be public because it is + * used in {@link CompoundBloomFilter} to avoid object creation on every + * Bloom filter lookup, but has to be used with caution. + * + * @return the buffer of this block for read-only operations + */ + public ByteBuffer getBufferReadOnly() { + return buf; + } + + /** + * Returns a byte buffer of this block, including header data, positioned at + * the beginning of header. The underlying data array is not copied. + * + * @return the byte buffer with header included + */ + public ByteBuffer getBufferWithHeader() { + ByteBuffer dupBuf = buf.duplicate(); + dupBuf.rewind(); + return dupBuf; + } + + /** + * Deserializes fields of the given writable using the data portion of this + * block. Does not check that all the block data has been read. + */ + public void readInto(Writable w) throws IOException { + Preconditions.checkNotNull(w); + + if (Writables.getWritable(buf.array(), buf.arrayOffset() + HEADER_SIZE, + buf.limit() - HEADER_SIZE, w) == null) { + throw new IOException("Failed to deserialize block " + this + " into a " + + w.getClass().getSimpleName()); + } + } + + private void sanityCheckAssertion(long valueFromBuf, long valueFromField, + String fieldName) throws IOException { + if (valueFromBuf != valueFromField) { + throw new AssertionError(fieldName + " in the buffer (" + valueFromBuf + + ") is different from that in the field (" + valueFromField + ")"); + } + } + + /** + * Checks if the block is internally consistent, i.e. the first + * {@link #HEADER_SIZE} bytes of the buffer contain a valid header consistent + * with the fields. This function is primary for testing and debugging, and + * is not thread-safe, because it alters the internal buffer pointer. + */ + void sanityCheck() throws IOException { + buf.rewind(); + + { + BlockType blockTypeFromBuf = BlockType.read(buf); + if (blockTypeFromBuf != blockType) { + throw new IOException("Block type stored in the buffer: " + + blockTypeFromBuf + ", block type field: " + blockType); + } + } + + sanityCheckAssertion(buf.getInt(), onDiskSizeWithoutHeader, + "onDiskSizeWithoutHeader"); + + sanityCheckAssertion(buf.getInt(), uncompressedSizeWithoutHeader, + "uncompressedSizeWithoutHeader"); + + sanityCheckAssertion(buf.getLong(), prevBlockOffset, "prevBlocKOffset"); + + int expectedBufLimit = uncompressedSizeWithoutHeader + HEADER_SIZE; + if (buf.limit() != expectedBufLimit) { + throw new AssertionError("Expected buffer limit " + expectedBufLimit + + ", got " + buf.limit()); + } + + // We might optionally allocate HEADER_SIZE more bytes to read the next + // block's, header, so there are two sensible values for buffer capacity. + if (buf.capacity() != uncompressedSizeWithoutHeader + HEADER_SIZE && + buf.capacity() != uncompressedSizeWithoutHeader + 2 * HEADER_SIZE) { + throw new AssertionError("Invalid buffer capacity: " + buf.capacity() + + ", expected " + (uncompressedSizeWithoutHeader + HEADER_SIZE) + + " or " + (uncompressedSizeWithoutHeader + 2 * HEADER_SIZE)); + } + } + + @Override + public String toString() { + return "blockType=" + + blockType + + ", onDiskSizeWithoutHeader=" + + onDiskSizeWithoutHeader + + ", uncompressedSizeWithoutHeader=" + + uncompressedSizeWithoutHeader + + ", prevBlockOffset=" + + prevBlockOffset + + ", dataBeginsWith=" + + Bytes.toStringBinary(buf.array(), buf.arrayOffset() + HEADER_SIZE, + Math.min(32, buf.limit() - buf.arrayOffset() - HEADER_SIZE)) + + ", fileOffset=" + offset; + } + + private void validateOnDiskSizeWithoutHeader( + int expectedOnDiskSizeWithoutHeader) throws IOException { + if (onDiskSizeWithoutHeader != expectedOnDiskSizeWithoutHeader) { + String blockInfoMsg = + "Block offset: " + offset + ", data starts with: " + + Bytes.toStringBinary(buf.array(), buf.arrayOffset(), + buf.arrayOffset() + Math.min(32, buf.limit())); + throw new IOException("On-disk size without header provided is " + + expectedOnDiskSizeWithoutHeader + ", but block " + + "header contains " + onDiskSizeWithoutHeader + ". " + + blockInfoMsg); + } + } + + /** + * Always allocates a new buffer of the correct size. Copies header bytes + * from the existing buffer. Does not change header fields. + * + * @param extraBytes whether to reserve room in the buffer to read the next + * block's header + */ + private void allocateBuffer(boolean extraBytes) { + int capacityNeeded = HEADER_SIZE + uncompressedSizeWithoutHeader + + (extraBytes ? HEADER_SIZE : 0); + + ByteBuffer newBuf = ByteBuffer.allocate(capacityNeeded); + + // Copy header bytes. + System.arraycopy(buf.array(), buf.arrayOffset(), newBuf.array(), + newBuf.arrayOffset(), HEADER_SIZE); + + buf = newBuf; + buf.limit(HEADER_SIZE + uncompressedSizeWithoutHeader); + } + + /** An additional sanity-check in case no compression is being used. */ + public void assumeUncompressed() throws IOException { + if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader) { + throw new IOException("Using no compression but " + + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", " + + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader); + } + } + + /** + * @param expectedType the expected type of this block + * @throws IOException if this block's type is different than expected + */ + public void expectType(BlockType expectedType) throws IOException { + if (blockType != expectedType) { + throw new IOException("Invalid block type: expected=" + expectedType + + ", actual=" + blockType); + } + } + + /** @return the offset of this block in the file it was read from */ + public long getOffset() { + if (offset < 0) { + throw new IllegalStateException( + "HFile block offset not initialized properly"); + } + return offset; + } + + /** + * @return a byte stream reading the data section of this block + */ + public DataInputStream getByteStream() { + return new DataInputStream(new ByteArrayInputStream(buf.array(), + buf.arrayOffset() + HEADER_SIZE, buf.limit() - HEADER_SIZE)); + } + + @Override + public long heapSize() { + // This object, block type and byte buffer reference, on-disk and + // uncompressed size, next block's on-disk size, offset and previous + // offset, byte buffer object, and its byte array. Might also need to add + // some fields inside the byte buffer. + return ClassSize.align(ClassSize.OBJECT + 2 * ClassSize.REFERENCE + 3 + * Bytes.SIZEOF_INT + 2 * Bytes.SIZEOF_LONG + BYTE_BUFFER_HEAP_SIZE) + + ClassSize.align(buf.capacity()); + } + + /** + * Read from an input stream. Analogous to + * {@link IOUtils#readFully(InputStream, byte[], int, int)}, but specifies a + * number of "extra" bytes that would be desirable but not absolutely + * necessary to read. + * + * @param in the input stream to read from + * @param buf the buffer to read into + * @param bufOffset the destination offset in the buffer + * @param necessaryLen the number of bytes that are absolutely necessary to + * read + * @param extraLen the number of extra bytes that would be nice to read + * @return true if succeeded reading the extra bytes + * @throws IOException if failed to read the necessary bytes + */ + public static boolean readWithExtra(InputStream in, byte buf[], + int bufOffset, int necessaryLen, int extraLen) throws IOException { + int bytesRemaining = necessaryLen + extraLen; + while (bytesRemaining > 0) { + int ret = in.read(buf, bufOffset, bytesRemaining); + if (ret == -1 && bytesRemaining <= extraLen) { + // We could not read the "extra data", but that is OK. + break; + } + + if (ret < 0) { + throw new IOException("Premature EOF from inputStream (read " + + "returned " + ret + ", was trying to read " + necessaryLen + + " necessary bytes and " + extraLen + " extra bytes, " + + "successfully read " + + (necessaryLen + extraLen - bytesRemaining)); + } + bufOffset += ret; + bytesRemaining -= ret; + } + return bytesRemaining <= 0; + } + + /** + * @return the on-disk size of the next block (including the header size) + * that was read by peeking into the next block's header + */ + public int getNextBlockOnDiskSizeWithHeader() { + return nextBlockOnDiskSizeWithHeader; + } + + + /** + * Unified version 2 {@link HFile} block writer. The intended usage pattern + * is as follows: + *
    + *
  • Construct an {@link HFileBlock.Writer}, providing a compression + * algorithm + *
  • Call {@link Writer#startWriting(BlockType)} and get a data stream to + * write to + *
  • Write your data into the stream + *
  • Call {@link Writer#writeHeaderAndData()} as many times as you need to + * store the serialized block into an external stream, or call + * {@link Writer#getHeaderAndData()} to get it as a byte array. + *
  • Repeat to write more blocks + *
+ *

+ */ + public static class Writer { + + private enum State { + INIT, + WRITING, + BLOCK_READY + }; + + /** Writer state. Used to ensure the correct usage protocol. */ + private State state = State.INIT; + + /** Compression algorithm for all blocks this instance writes. */ + private final Compression.Algorithm compressAlgo; + + /** + * The stream we use to accumulate data in the on-disk format for each + * block (i.e. compressed data, or uncompressed if using no compression). + * We reset this stream at the end of each block and reuse it. The header + * is written as the first {@link #HEADER_SIZE} bytes into this stream. + */ + private ByteArrayOutputStream baosOnDisk; + + /** + * The stream we use to accumulate uncompressed block data for + * cache-on-write. Null when cache-on-write is turned off. + */ + private ByteArrayOutputStream baosInMemory; + + /** Compressor, which is also reused between consecutive blocks. */ + private Compressor compressor; + + /** Current block type. Set in {@link #startWriting(BlockType)}. */ + private BlockType blockType; + + /** + * A stream that we write uncompressed bytes to, which compresses them and + * writes them to {@link #baosOnDisk}. + */ + private DataOutputStream userDataStream; + + /** + * Bytes to be written to the file system, including the header. Compressed + * if compression is turned on. + */ + private byte[] onDiskBytesWithHeader; + + /** + * The total number of uncompressed bytes written into the current block, + * with header size not included. Valid in the READY state. + */ + private int uncompressedSizeWithoutHeader; + + /** + * Only used when we are using cache-on-write. Valid in the READY state. + * Contains the header and the uncompressed bytes, so the length is + * {@link #uncompressedSizeWithoutHeader} + {@link HFileBlock#HEADER_SIZE}. + */ + private byte[] uncompressedBytesWithHeader; + + /** + * Current block's start offset in the {@link HFile}. Set in + * {@link #writeHeaderAndData(FSDataOutputStream)}. + */ + private long startOffset; + + /** + * Offset of previous block by block type. Updated when the next block is + * started. + */ + private long[] prevOffsetByType; + + /** + * Whether we are accumulating uncompressed bytes for the purpose of + * caching on write. + */ + private boolean cacheOnWrite; + + /** The offset of the previous block of the same type */ + private long prevOffset; + + /** + * @param blockType + * block type to create + * @param compressionAlgorithm + * compression algorithm to use + */ + public Writer(Compression.Algorithm compressionAlgorithm) { + compressAlgo = compressionAlgorithm == null ? NONE + : compressionAlgorithm; + + baosOnDisk = new ByteArrayOutputStream(); + if (compressAlgo != NONE) + compressor = compressionAlgorithm.getCompressor(); + + prevOffsetByType = new long[BlockType.values().length]; + for (int i = 0; i < prevOffsetByType.length; ++i) + prevOffsetByType[i] = -1; + } + + /** + * Starts writing into the block. The previous block's data is discarded. + * + * @return the stream the user can write their data into + * @throws IOException + */ + public DataOutputStream startWriting(BlockType newBlockType, + boolean cacheOnWrite) throws IOException { + if (state == State.BLOCK_READY && startOffset != -1) { + // We had a previous block that was written to a stream at a specific + // offset. Save that offset as the last offset of a block of that type. + prevOffsetByType[blockType.ordinal()] = startOffset; + } + + this.cacheOnWrite = cacheOnWrite; + + startOffset = -1; + blockType = newBlockType; + + baosOnDisk.reset(); + baosOnDisk.write(DUMMY_HEADER); + + state = State.WRITING; + if (compressAlgo == NONE) { + // We do not need a compression stream or a second uncompressed stream + // for cache-on-write. + userDataStream = new DataOutputStream(baosOnDisk); + } else { + OutputStream compressingOutputStream = + compressAlgo.createCompressionStream(baosOnDisk, compressor, 0); + + if (cacheOnWrite) { + // We save uncompressed data in a cache-on-write mode. + if (baosInMemory == null) + baosInMemory = new ByteArrayOutputStream(); + baosInMemory.reset(); + baosInMemory.write(DUMMY_HEADER); + userDataStream = new DataOutputStream(new DoubleOutputStream( + compressingOutputStream, baosInMemory)); + } else { + userDataStream = new DataOutputStream(compressingOutputStream); + } + } + + return userDataStream; + } + + /** + * Returns the stream for the user to write to. The block writer takes care + * of handling compression and buffering for caching on write. Can only be + * called in the "writing" state. + * + * @return the data output stream for the user to write to + */ + DataOutputStream getUserDataStream() { + expectState(State.WRITING); + return userDataStream; + } + + /** + * Transitions the block writer from the "writing" state to the "block + * ready" state. Does nothing if a block is already finished. + */ + private void ensureBlockReady() throws IOException { + Preconditions.checkState(state != State.INIT, + "Unexpected state: " + state); + + if (state == State.BLOCK_READY) + return; + + finishBlock(); + state = State.BLOCK_READY; + } + + /** + * An internal method that flushes the compressing stream (if using + * compression), serializes the header, and takes care of the separate + * uncompressed stream for caching on write, if applicable. Block writer + * state transitions must be managed by the caller. + */ + private void finishBlock() throws IOException { + userDataStream.flush(); + uncompressedSizeWithoutHeader = userDataStream.size(); + + onDiskBytesWithHeader = baosOnDisk.toByteArray(); + prevOffset = prevOffsetByType[blockType.ordinal()]; + putHeader(onDiskBytesWithHeader, 0); + + if (cacheOnWrite && compressAlgo != NONE) { + uncompressedBytesWithHeader = baosInMemory.toByteArray(); + + if (uncompressedSizeWithoutHeader != + uncompressedBytesWithHeader.length - HEADER_SIZE) { + throw new IOException("Uncompressed size mismatch: " + + uncompressedSizeWithoutHeader + " vs. " + + (uncompressedBytesWithHeader.length - HEADER_SIZE)); + } + + // Write the header into the beginning of the uncompressed byte array. + putHeader(uncompressedBytesWithHeader, 0); + } + } + + /** Put the header into the given byte array at the given offset. */ + private void putHeader(byte[] dest, int offset) { + offset = blockType.put(dest, offset); + offset = Bytes.putInt(dest, offset, onDiskBytesWithHeader.length + - HEADER_SIZE); + offset = Bytes.putInt(dest, offset, uncompressedSizeWithoutHeader); + Bytes.putLong(dest, offset, prevOffset); + } + + /** + * Similar to {@link #writeHeaderAndData(DataOutputStream)}, but records + * the offset of this block so that it can be referenced in the next block + * of the same type. + * + * @param out + * @throws IOException + */ + public void writeHeaderAndData(FSDataOutputStream out) throws IOException { + long offset = out.getPos(); + if (startOffset != -1 && offset != startOffset) { + throw new IOException("A " + blockType + " block written to a " + + "stream twice, first at offset " + startOffset + ", then at " + + offset); + } + startOffset = offset; + + writeHeaderAndData((DataOutputStream) out); + } + + /** + * Writes the header and the compressed data of this block (or uncompressed + * data when not using compression) into the given stream. Can be called in + * the "writing" state or in the "block ready" state. If called in the + * "writing" state, transitions the writer to the "block ready" state. + * + * @param out the output stream to write the + * @throws IOException + */ + private void writeHeaderAndData(DataOutputStream out) throws IOException { + ensureBlockReady(); + out.write(onDiskBytesWithHeader); + } + + /** + * Returns the header or the compressed data (or uncompressed data when not + * using compression) as a byte array. Can be called in the "writing" state + * or in the "block ready" state. If called in the "writing" state, + * transitions the writer to the "block ready" state. + * + * @return header and data as they would be stored on disk in a byte array + * @throws IOException + */ + public byte[] getHeaderAndData() throws IOException { + ensureBlockReady(); + return onDiskBytesWithHeader; + } + + /** + * Releases the compressor this writer uses to compress blocks into the + * compressor pool. Needs to be called before the writer is discarded. + */ + public void releaseCompressor() { + if (compressor != null) { + compressAlgo.returnCompressor(compressor); + compressor = null; + } + } + + /** + * Returns the on-disk size of the data portion of the block. This is the + * compressed size if compression is enabled. Can only be called in the + * "block ready" state. Header is not compressed, and its size is not + * included in the return value. + * + * @return the on-disk size of the block, not including the header. + */ + public int getOnDiskSizeWithoutHeader() { + expectState(State.BLOCK_READY); + return onDiskBytesWithHeader.length - HEADER_SIZE; + } + + /** + * Returns the on-disk size of the block. Can only be called in the + * "block ready" state. + * + * @return the on-disk size of the block ready to be written, including the + * header size + */ + public int getOnDiskSizeWithHeader() { + expectState(State.BLOCK_READY); + return onDiskBytesWithHeader.length; + } + + /** + * The uncompressed size of the block data. Does not include header size. + */ + public int getUncompressedSizeWithoutHeader() { + expectState(State.BLOCK_READY); + return uncompressedSizeWithoutHeader; + } + + /** + * The uncompressed size of the block data, including header size. + */ + public int getUncompressedSizeWithHeader() { + expectState(State.BLOCK_READY); + return uncompressedSizeWithoutHeader + HEADER_SIZE; + } + + /** @return true if a block is being written */ + public boolean isWriting() { + return state == State.WRITING; + } + + /** + * Returns the number of bytes written into the current block so far, or + * zero if not writing the block at the moment. Note that this will return + * zero in the "block ready" state as well. + * + * @return the number of bytes written + */ + public int blockSizeWritten() { + if (state != State.WRITING) + return 0; + return userDataStream.size(); + } + + /** + * Returns the header followed by the uncompressed data, even if using + * compression. This is needed for storing uncompressed blocks in the block + * cache. Can be called in the "writing" state or the "block ready" state. + * + * @return uncompressed block bytes for caching on write + */ + private byte[] getUncompressedDataWithHeader() { + expectState(State.BLOCK_READY); + + if (compressAlgo == NONE) + return onDiskBytesWithHeader; + + if (!cacheOnWrite) + throw new IllegalStateException("Cache-on-write is turned off"); + + if (uncompressedBytesWithHeader == null) + throw new NullPointerException(); + + return uncompressedBytesWithHeader; + } + + private void expectState(State expectedState) { + if (state != expectedState) { + throw new IllegalStateException("Expected state: " + expectedState + + ", actual state: " + state); + } + } + + /** + * Similar to {@link #getUncompressedDataWithHeader()} but returns a byte + * buffer. + * + * @return uncompressed block for caching on write in the form of a buffer + */ + public ByteBuffer getUncompressedBufferWithHeader() { + byte[] b = getUncompressedDataWithHeader(); + return ByteBuffer.wrap(b, 0, b.length); + } + + /** + * Takes the given {@link BlockWritable} instance, creates a new block of + * its appropriate type, writes the writable into this block, and flushes + * the block into the output stream. The writer is instructed not to buffer + * uncompressed bytes for cache-on-write. + * + * @param bw the block-writable object to write as a block + * @param out the file system output stream + * @throws IOException + */ + public void writeBlock(BlockWritable bw, FSDataOutputStream out) + throws IOException { + bw.writeToBlock(startWriting(bw.getBlockType(), false)); + writeHeaderAndData(out); + } + + public HFileBlock getBlockForCaching() { + return new HFileBlock(blockType, onDiskBytesWithHeader.length + - HEADER_SIZE, uncompressedSizeWithoutHeader, prevOffset, + getUncompressedBufferWithHeader(), false, startOffset); + } + + } + + /** Something that can be written into a block. */ + public interface BlockWritable { + + /** The type of block this data should use. */ + BlockType getBlockType(); + + /** + * Writes the block to the provided stream. Must not write any magic + * records. + * + * @param out a stream to write uncompressed data into + */ + void writeToBlock(DataOutput out) throws IOException; + } + + // Block readers and writers + + /** An interface allowing to iterate {@link HFileBlock}s. */ + public interface BlockIterator { + + /** + * Get the next block, or null if there are no more blocks to iterate. + */ + HFileBlock nextBlock() throws IOException; + + /** + * Similar to {@link #nextBlock()} but checks block type, throws an + * exception if incorrect, and returns the data portion of the block as + * an input stream. + */ + DataInputStream nextBlockAsStream(BlockType blockType) throws IOException; + } + + /** + * Just the basic ability to read blocks, providing optional hints of + * on-disk-size and/or uncompressed size. + */ + public interface BasicReader { + /** + * Reads the block at the given offset in the file with the given on-disk + * size and uncompressed size. + * + * @param offset + * @param onDiskSize the on-disk size of the entire block, including all + * applicable headers, or -1 if unknown + * @param uncompressedSize the uncompressed size of the compressed part of + * the block, or -1 if unknown + * @return the newly read block + */ + HFileBlock readBlockData(long offset, long onDiskSize, + int uncompressedSize, boolean pread) throws IOException; + } + + /** A full-fledged reader with an iteration ability. */ + public interface FSReader extends BasicReader { + + /** + * Creates a block iterator over the given portion of the {@link HFile}. + * The iterator returns blocks starting with offset such that offset <= + * startOffset < endOffset. + * + * @param startOffset the offset of the block to start iteration with + * @param endOffset the offset to end iteration at (exclusive) + * @return an iterator of blocks between the two given offsets + */ + BlockIterator blockRange(long startOffset, long endOffset); + } + + /** + * A common implementation of some methods of {@link FSReader} and some + * tools for implementing HFile format version-specific block readers. + */ + public abstract static class AbstractFSReader implements FSReader { + + /** The file system stream of the underlying {@link HFile} */ + protected FSDataInputStream istream; + + /** Compression algorithm used by the {@link HFile} */ + protected Compression.Algorithm compressAlgo; + + /** The size of the file we are reading from, or -1 if unknown. */ + protected long fileSize; + + /** The default buffer size for our buffered streams */ + public static final int DEFAULT_BUFFER_SIZE = 1 << 20; + + public AbstractFSReader(FSDataInputStream istream, Algorithm compressAlgo, + long fileSize) { + this.istream = istream; + this.compressAlgo = compressAlgo; + this.fileSize = fileSize; + } + + @Override + public BlockIterator blockRange(final long startOffset, + final long endOffset) { + return new BlockIterator() { + private long offset = startOffset; + + @Override + public HFileBlock nextBlock() throws IOException { + if (offset >= endOffset) + return null; + HFileBlock b = readBlockData(offset, -1, -1, false); + offset += b.getOnDiskSizeWithHeader(); + return b; + } + + @Override + public DataInputStream nextBlockAsStream(BlockType blockType) + throws IOException { + HFileBlock blk = nextBlock(); + if (blk.getBlockType() != blockType) { + throw new IOException("Expected block of type " + blockType + + " but found " + blk.getBlockType()); + } + return blk.getByteStream(); + } + }; + } + + /** + * Does a positional read or a seek and read into the given buffer. Returns + * the on-disk size of the next block, or -1 if it could not be determined. + * + * @param dest destination buffer + * @param destOffset offset in the destination buffer + * @param size size of the block to be read + * @param peekIntoNextBlock whether to read the next block's on-disk size + * @param fileOffset position in the stream to read at + * @param pread whether we should do a positional read + * @return the on-disk size of the next block with header size included, or + * -1 if it could not be determined + * @throws IOException + */ + protected int readAtOffset(byte[] dest, int destOffset, int size, + boolean peekIntoNextBlock, long fileOffset, boolean pread) + throws IOException { + if (peekIntoNextBlock && + destOffset + size + HEADER_SIZE > dest.length) { + // We are asked to read the next block's header as well, but there is + // not enough room in the array. + throw new IOException("Attempted to read " + size + " bytes and " + + HEADER_SIZE + " bytes of next header into a " + dest.length + + "-byte array at offset " + destOffset); + } + + if (pread) { + // Positional read. Better for random reads. + int extraSize = peekIntoNextBlock ? HEADER_SIZE : 0; + + int ret = istream.read(fileOffset, dest, destOffset, size + extraSize); + if (ret < size) { + throw new IOException("Positional read of " + size + " bytes " + + "failed at offset " + fileOffset + " (returned " + ret + ")"); + } + + if (ret == size || ret < size + extraSize) { + // Could not read the next block's header, or did not try. + return -1; + } + } else { + // Seek + read. Better for scanning. + synchronized (istream) { + istream.seek(fileOffset); + + long realOffset = istream.getPos(); + if (realOffset != fileOffset) { + throw new IOException("Tried to seek to " + fileOffset + " to " + + "read " + size + " bytes, but pos=" + realOffset + + " after seek"); + } + + if (!peekIntoNextBlock) { + IOUtils.readFully(istream, dest, destOffset, size); + return -1; + } + + // Try to read the next block header. + if (!readWithExtra(istream, dest, destOffset, size, HEADER_SIZE)) + return -1; + } + } + + assert peekIntoNextBlock; + return Bytes.toInt(dest, destOffset + size + BlockType.MAGIC_LENGTH) + + HEADER_SIZE; + } + + /** + * Decompresses data from the given stream using the configured compression + * algorithm. + * + * @param boundedStream + * a stream to read compressed data from, bounded to the exact + * amount of compressed data + * @param compressedSize + * compressed data size, header not included + * @param uncompressedSize + * uncompressed data size, header not included + * @param header + * the header to include before the decompressed data, or null. + * Only the first {@link HFileBlock#HEADER_SIZE} bytes of the + * buffer are included. + * @return the byte buffer containing the given header (optionally) and the + * decompressed data + * @throws IOException + */ + protected void decompress(byte[] dest, int destOffset, + InputStream bufferedBoundedStream, int compressedSize, + int uncompressedSize) throws IOException { + Decompressor decompressor = null; + try { + decompressor = compressAlgo.getDecompressor(); + InputStream is = compressAlgo.createDecompressionStream( + bufferedBoundedStream, decompressor, 0); + + IOUtils.readFully(is, dest, destOffset, uncompressedSize); + is.close(); + } finally { + if (decompressor != null) { + compressAlgo.returnDecompressor(decompressor); + } + } + } + + /** + * Creates a buffered stream reading a certain slice of the file system + * input stream. We need this because the decompression we use seems to + * expect the input stream to be bounded. + * + * @param offset the starting file offset the bounded stream reads from + * @param size the size of the segment of the file the stream should read + * @param pread whether to use position reads + * @return a stream restricted to the given portion of the file + */ + protected InputStream createBufferedBoundedStream(long offset, + int size, boolean pread) { + return new BufferedInputStream(new BoundedRangeFileInputStream(istream, + offset, size, pread), Math.min(DEFAULT_BUFFER_SIZE, size)); + } + + } + + /** + * Reads version 1 blocks from the file system. In version 1 blocks, + * everything is compressed, including the magic record, if compression is + * enabled. Everything might be uncompressed if no compression is used. This + * reader returns blocks represented in the uniform version 2 format in + * memory. + */ + public static class FSReaderV1 extends AbstractFSReader { + + /** Header size difference between version 1 and 2 */ + private static final int HEADER_DELTA = HEADER_SIZE - MAGIC_LENGTH; + + public FSReaderV1(FSDataInputStream istream, Algorithm compressAlgo, + long fileSize) { + super(istream, compressAlgo, fileSize); + } + + /** + * Read a version 1 block. There is no uncompressed header, and the block + * type (the magic record) is part of the compressed data. This + * implementation assumes that the bounded range file input stream is + * needed to stop the decompressor reading into next block, because the + * decompressor just grabs a bunch of data without regard to whether it is + * coming to end of the compressed section. + * + * The block returned is still a version 2 block, and in particular, its + * first {@link #HEADER_SIZE} bytes contain a valid version 2 header. + * + * @param offset the offset of the block to read in the file + * @param onDiskSizeWithMagic the on-disk size of the version 1 block, + * including the magic record, which is the part of compressed + * data if using compression + * @param uncompressedSizeWithMagic uncompressed size of the version 1 + * block, including the magic record + */ + @Override + public HFileBlock readBlockData(long offset, long onDiskSizeWithMagic, + int uncompressedSizeWithMagic, boolean pread) throws IOException { + if (uncompressedSizeWithMagic <= 0) { + throw new IOException("Invalid uncompressedSize=" + + uncompressedSizeWithMagic + " for a version 1 block"); + } + + if (onDiskSizeWithMagic <= 0 || onDiskSizeWithMagic >= Integer.MAX_VALUE) + { + throw new IOException("Invalid onDiskSize=" + onDiskSizeWithMagic + + " (maximum allowed: " + Integer.MAX_VALUE + ")"); + } + + int onDiskSize = (int) onDiskSizeWithMagic; + + if (uncompressedSizeWithMagic < MAGIC_LENGTH) { + throw new IOException("Uncompressed size for a version 1 block is " + + uncompressedSizeWithMagic + " but must be at least " + + MAGIC_LENGTH); + } + + // The existing size already includes magic size, and we are inserting + // a version 2 header. + ByteBuffer buf = ByteBuffer.allocate(uncompressedSizeWithMagic + + HEADER_DELTA); + + int onDiskSizeWithoutHeader; + if (compressAlgo == Compression.Algorithm.NONE) { + // A special case when there is no compression. + if (onDiskSize != uncompressedSizeWithMagic) { + throw new IOException("onDiskSize=" + onDiskSize + + " and uncompressedSize=" + uncompressedSizeWithMagic + + " must be equal for version 1 with no compression"); + } + + // The first MAGIC_LENGTH bytes of what this will read will be + // overwritten. + readAtOffset(buf.array(), buf.arrayOffset() + HEADER_DELTA, + onDiskSize, false, offset, pread); + + onDiskSizeWithoutHeader = uncompressedSizeWithMagic - MAGIC_LENGTH; + } else { + InputStream bufferedBoundedStream = createBufferedBoundedStream( + offset, onDiskSize, pread); + decompress(buf.array(), buf.arrayOffset() + HEADER_DELTA, + bufferedBoundedStream, onDiskSize, uncompressedSizeWithMagic); + + // We don't really have a good way to exclude the "magic record" size + // from the compressed block's size, since it is compressed as well. + onDiskSizeWithoutHeader = onDiskSize; + } + + BlockType newBlockType = BlockType.parse(buf.array(), buf.arrayOffset() + + HEADER_DELTA, MAGIC_LENGTH); + + // We set the uncompressed size of the new HFile block we are creating + // to the size of the data portion of the block without the magic record, + // since the magic record gets moved to the header. + HFileBlock b = new HFileBlock(newBlockType, onDiskSizeWithoutHeader, + uncompressedSizeWithMagic - MAGIC_LENGTH, -1L, buf, true, offset); + return b; + } + } + + /** + * We always prefetch the header of the next block, so that we know its + * on-disk size in advance and can read it in one operation. + */ + private static class PrefetchedHeader { + long offset = -1; + byte[] header = new byte[HEADER_SIZE]; + ByteBuffer buf = ByteBuffer.wrap(header, 0, HEADER_SIZE); + } + + /** Reads version 2 blocks from the filesystem. */ + public static class FSReaderV2 extends AbstractFSReader { + + private ThreadLocal prefetchedHeaderForThread = + new ThreadLocal() { + @Override + public PrefetchedHeader initialValue() { + return new PrefetchedHeader(); + } + }; + + public FSReaderV2(FSDataInputStream istream, Algorithm compressAlgo, + long fileSize) { + super(istream, compressAlgo, fileSize); + } + + /** + * Reads a version 2 block. Tries to do as little memory allocation as + * possible, using the provided on-disk size. + * + * @param offset the offset in the stream to read at + * @param onDiskSizeWithHeaderL the on-disk size of the block, including + * the header, or -1 if unknown + * @param uncompressedSize the uncompressed size of the the block. Always + * expected to be -1. This parameter is only used in version 1. + * @param pread whether to use a positional read + */ + @Override + public HFileBlock readBlockData(long offset, long onDiskSizeWithHeaderL, + int uncompressedSize, boolean pread) throws IOException { + if (offset < 0) { + throw new IOException("Invalid offset=" + offset + " trying to read " + + "block (onDiskSize=" + onDiskSizeWithHeaderL + + ", uncompressedSize=" + uncompressedSize + ")"); + } + if (uncompressedSize != -1) { + throw new IOException("Version 2 block reader API does not need " + + "the uncompressed size parameter"); + } + + if ((onDiskSizeWithHeaderL < HEADER_SIZE && onDiskSizeWithHeaderL != -1) + || onDiskSizeWithHeaderL >= Integer.MAX_VALUE) { + throw new IOException("Invalid onDisksize=" + onDiskSizeWithHeaderL + + ": expected to be at least " + HEADER_SIZE + + " and at most " + Integer.MAX_VALUE + ", or -1 (offset=" + + offset + ", uncompressedSize=" + uncompressedSize + ")"); + } + + int onDiskSizeWithHeader = (int) onDiskSizeWithHeaderL; + + HFileBlock b; + if (onDiskSizeWithHeader > 0) { + // We know the total on-disk size but not the uncompressed size. Read + // the entire block into memory, then parse the header and decompress + // from memory if using compression. This code path is used when + // doing a random read operation relying on the block index, as well as + // when the client knows the on-disk size from peeking into the next + // block's header (e.g. this block's header) when reading the previous + // block. This is the faster and more preferable case. + + int onDiskSizeWithoutHeader = onDiskSizeWithHeader - HEADER_SIZE; + assert onDiskSizeWithoutHeader >= 0; + + // See if we can avoid reading the header. This is desirable, because + // we will not incur a seek operation to seek back if we have already + // read this block's header as part of the previous read's look-ahead. + PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get(); + byte[] header = prefetchedHeader.offset == offset + ? prefetchedHeader.header : null; + + // Size that we have to skip in case we have already read the header. + int preReadHeaderSize = header == null ? 0 : HEADER_SIZE; + + if (compressAlgo == Compression.Algorithm.NONE) { + // Just read the whole thing. Allocate enough space to read the + // next block's header too. + + ByteBuffer headerAndData = ByteBuffer.allocate(onDiskSizeWithHeader + + HEADER_SIZE); + headerAndData.limit(onDiskSizeWithHeader); + + if (header != null) { + System.arraycopy(header, 0, headerAndData.array(), 0, + HEADER_SIZE); + } + + int nextBlockOnDiskSizeWithHeader = readAtOffset( + headerAndData.array(), headerAndData.arrayOffset() + + preReadHeaderSize, onDiskSizeWithHeader + - preReadHeaderSize, true, offset + preReadHeaderSize, + pread); + + b = new HFileBlock(headerAndData); + b.assumeUncompressed(); + b.validateOnDiskSizeWithoutHeader(onDiskSizeWithoutHeader); + b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSizeWithHeader; + + if (b.nextBlockOnDiskSizeWithHeader > 0) + setNextBlockHeader(offset, b); + } else { + // Allocate enough space to fit the next block's header too. + byte[] onDiskBlock = new byte[onDiskSizeWithHeader + HEADER_SIZE]; + + int nextBlockOnDiskSize = readAtOffset(onDiskBlock, + preReadHeaderSize, onDiskSizeWithHeader - preReadHeaderSize, + true, offset + preReadHeaderSize, pread); + + if (header == null) + header = onDiskBlock; + + try { + b = new HFileBlock(ByteBuffer.wrap(header, 0, HEADER_SIZE)); + } catch (IOException ex) { + // Seen in load testing. Provide comprehensive debug info. + throw new IOException("Failed to read compressed block at " + + offset + ", onDiskSizeWithoutHeader=" + onDiskSizeWithHeader + + ", preReadHeaderSize=" + preReadHeaderSize + + ", header.length=" + header.length + ", header bytes: " + + Bytes.toStringBinary(header, 0, HEADER_SIZE), ex); + } + b.validateOnDiskSizeWithoutHeader(onDiskSizeWithoutHeader); + b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSize; + + DataInputStream dis = new DataInputStream(new ByteArrayInputStream( + onDiskBlock, HEADER_SIZE, onDiskSizeWithoutHeader)); + + // This will allocate a new buffer but keep header bytes. + b.allocateBuffer(b.nextBlockOnDiskSizeWithHeader > 0); + + decompress(b.buf.array(), b.buf.arrayOffset() + HEADER_SIZE, dis, + onDiskSizeWithoutHeader, b.uncompressedSizeWithoutHeader); + + // Copy next block's header bytes into the new block if we have them. + if (nextBlockOnDiskSize > 0) { + System.arraycopy(onDiskBlock, onDiskSizeWithHeader, b.buf.array(), + b.buf.arrayOffset() + HEADER_SIZE + + b.uncompressedSizeWithoutHeader, HEADER_SIZE); + + setNextBlockHeader(offset, b); + } + } + + } else { + // We don't know the on-disk size. Read the header first, determine the + // on-disk size from it, and read the remaining data, thereby incurring + // two read operations. This might happen when we are doing the first + // read in a series of reads or a random read, and we don't have access + // to the block index. This is costly and should happen very rarely. + + // Check if we have read this block's header as part of reading the + // previous block. If so, don't read the header again. + PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get(); + ByteBuffer headerBuf = prefetchedHeader.offset == offset ? + prefetchedHeader.buf : null; + + if (headerBuf == null) { + // Unfortunately, we still have to do a separate read operation to + // read the header. + headerBuf = ByteBuffer.allocate(HEADER_SIZE);; + readAtOffset(headerBuf.array(), headerBuf.arrayOffset(), HEADER_SIZE, + false, offset, pread); + } + + b = new HFileBlock(headerBuf); + + // This will also allocate enough room for the next block's header. + b.allocateBuffer(true); + + if (compressAlgo == Compression.Algorithm.NONE) { + + // Avoid creating bounded streams and using a "codec" that does + // nothing. + b.assumeUncompressed(); + b.nextBlockOnDiskSizeWithHeader = readAtOffset(b.buf.array(), + b.buf.arrayOffset() + HEADER_SIZE, + b.uncompressedSizeWithoutHeader, true, offset + HEADER_SIZE, + pread); + + if (b.nextBlockOnDiskSizeWithHeader > 0) { + setNextBlockHeader(offset, b); + } + } else { + // Allocate enough space for the block's header and compressed data. + byte[] compressedBytes = new byte[b.getOnDiskSizeWithHeader() + + HEADER_SIZE]; + + b.nextBlockOnDiskSizeWithHeader = readAtOffset(compressedBytes, + HEADER_SIZE, b.onDiskSizeWithoutHeader, true, offset + + HEADER_SIZE, pread); + DataInputStream dis = new DataInputStream(new ByteArrayInputStream( + compressedBytes, HEADER_SIZE, b.onDiskSizeWithoutHeader)); + + decompress(b.buf.array(), b.buf.arrayOffset() + HEADER_SIZE, dis, + b.onDiskSizeWithoutHeader, b.uncompressedSizeWithoutHeader); + + if (b.nextBlockOnDiskSizeWithHeader > 0) { + // Copy the next block's header into the new block. + int nextHeaderOffset = b.buf.arrayOffset() + HEADER_SIZE + + b.uncompressedSizeWithoutHeader; + System.arraycopy(compressedBytes, + compressedBytes.length - HEADER_SIZE, + b.buf.array(), + nextHeaderOffset, + HEADER_SIZE); + + setNextBlockHeader(offset, b); + } + } + } + + b.offset = offset; + return b; + } + + private void setNextBlockHeader(long offset, HFileBlock b) { + PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get(); + prefetchedHeader.offset = offset + b.getOnDiskSizeWithHeader(); + int nextHeaderOffset = b.buf.arrayOffset() + HEADER_SIZE + + b.uncompressedSizeWithoutHeader; + System.arraycopy(b.buf.array(), nextHeaderOffset, + prefetchedHeader.header, 0, HEADER_SIZE); + } + + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java new file mode 100644 index 0000000..953896e --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java @@ -0,0 +1,1299 @@ +/* + * Copyright 2011 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.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.HeapSize; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter; +import org.apache.hadoop.io.RawComparator; +import org.apache.hadoop.io.WritableUtils; + +/** + * Provides functionality to write ({@link BlockIndexWriter}) and read + * ({@link BlockIndexReader}) single-level and multi-level block indexes. + * + * Examples of how to use the block index writer can be found in + * {@link CompoundBloomFilterWriter} and {@link HFileWriterV2}. Examples of how + * to use the reader can be found in {@link HFileReaderV2} and + * {@link TestHFileBlockIndex}. + */ +public class HFileBlockIndex { + + private static final Log LOG = LogFactory.getLog(HFileBlockIndex.class); + + static final int DEFAULT_MAX_CHUNK_SIZE = 128 * 1024; + + /** + * The maximum size guideline for index blocks (both leaf, intermediate, and + * root). If not specified, {@link #DEFAULT_MAX_CHUNK_SIZE} is used. + */ + public static final String MAX_CHUNK_SIZE_KEY = "hfile.index.block.max.size"; + + /** + * The number of bytes stored in each "secondary index" entry in addition to + * key bytes in the non-root index block format. The first long is the file + * offset of the deeper-level block the entry points to, and the int that + * follows is that block's on-disk size without including header. + */ + static final int SECONDARY_INDEX_ENTRY_OVERHEAD = Bytes.SIZEOF_INT + + Bytes.SIZEOF_LONG; + + /** + * Error message when trying to use inline block API in single-level mode. + */ + private static final String INLINE_BLOCKS_NOT_ALLOWED = + "Inline blocks are not allowed in the single-level-only mode"; + + /** + * Configuration key to cache leaf- and intermediate-level index blocks on + * write. + */ + public static final String CACHE_INDEX_BLOCKS_ON_WRITE_KEY = + "hfile.block.index.cacheonwrite"; + + /** + * The size of a meta-data record used for finding the mid-key in a + * multi-level index. Consists of the middle leaf-level index block offset + * (long), its on-disk size without header included (int), and the mid-key + * entry's zero-based index in that leaf index block. + */ + private static final int MID_KEY_METADATA_SIZE = Bytes.SIZEOF_LONG + + 2 * Bytes.SIZEOF_INT; + + /** + * The reader will always hold the root level index in the memory. Index + * blocks at all other levels will be cached in the LRU cache in practice, + * although this API does not enforce that. + * + * All non-root (leaf and intermediate) index blocks contain what we call a + * "secondary index": an array of offsets to the entries within the block. + * This allows us to do binary search for the entry corresponding to the + * given key without having to deserialize the block. + */ + public static class BlockIndexReader implements HeapSize { + /** Needed doing lookup on blocks. */ + private final RawComparator comparator; + + // Root-level data. + private byte[][] blockKeys; + private long[] blockOffsets; + private int[] blockDataSizes; + private int rootByteSize = 0; + private int rootCount = 0; + + // Mid-key metadata. + private long midLeafBlockOffset = -1; + private int midLeafBlockOnDiskSize = -1; + private int midKeyEntry = -1; + + /** Pre-computed mid-key */ + private AtomicReference midKey = new AtomicReference(); + + /** + * The number of levels in the block index tree. One if there is only root + * level, two for root and leaf levels, etc. + */ + private int searchTreeLevel; + + /** A way to read {@link HFile} blocks at a given offset */ + private HFileBlock.BasicReader blockReader; + + public BlockIndexReader(final RawComparator c, final int treeLevel, + final HFileBlock.BasicReader blockReader) { + this(c, treeLevel); + this.blockReader = blockReader; + } + + public BlockIndexReader(final RawComparator c, final int treeLevel) + { + comparator = c; + searchTreeLevel = treeLevel; + } + + /** + * @return true if the block index is empty. + */ + public boolean isEmpty() { + return blockKeys.length == 0; + } + + /** + * Verifies that the block index is non-empty and throws an + * {@link IllegalStateException} otherwise. + */ + public void ensureNonEmpty() { + if (blockKeys.length == 0) { + throw new IllegalStateException("Block index is empty or not loaded"); + } + } + + /** + * Return the data block which contains this key. This function will only + * be called when the HFile version is larger than 1. + * + * @param key the key we are looking for + * @param keyOffset the offset of the key in its byte array + * @param keyLength the length of the key + * @param currentBlock the current block, to avoid re-reading the same + * block + * @return reader a basic way to load blocks + * @throws IOException + */ + public HFileBlock seekToDataBlock(final byte[] key, int keyOffset, + int keyLength, HFileBlock currentBlock) + throws IOException { + int rootLevelIndex = rootBlockContainingKey(key, keyOffset, keyLength); + if (rootLevelIndex < 0 || rootLevelIndex >= blockOffsets.length) { + return null; + } + + // Read the next-level (intermediate or leaf) index block. + long currentOffset = blockOffsets[rootLevelIndex]; + int currentOnDiskSize = blockDataSizes[rootLevelIndex]; + + int lookupLevel = 1; // How many levels deep we are in our lookup. + HFileBlock block = blockReader.readBlockData(currentOffset, + currentOnDiskSize, -1, true); + if (block == null) { + throw new IOException("Failed to read block at offset " + + currentOffset + ", onDiskSize=" + currentOnDiskSize); + } + while (!block.getBlockType().equals(BlockType.DATA)) { + // Read the block. It may be intermediate level block, leaf level block + // or data block. In any case, we expect non-root index block format. + + // We don't allow more than searchTreeLevel iterations of this loop. + if (++lookupLevel > searchTreeLevel) { + throw new IOException("Search Tree Level overflow: lookupLevel: "+ + lookupLevel + " searchTreeLevel: " + searchTreeLevel); + } + + // read to the byte buffer + ByteBuffer buffer = block.getBufferWithoutHeader(); + if (!locateNonRootIndexEntry(buffer, key, keyOffset, keyLength, + comparator)) { + throw new IOException("The key " + + Bytes.toStringBinary(key, keyOffset, keyLength) + + " is before the" + " first key of the non-root index block " + + block); + } + + currentOffset = buffer.getLong(); + currentOnDiskSize = buffer.getInt(); + + // Located a deeper-level block, now read it. + if (currentBlock != null && currentBlock.getOffset() == currentOffset) + { + // Avoid reading the same block. + block = currentBlock; + } else { + block = blockReader.readBlockData(currentOffset, currentOnDiskSize, + -1, true); + } + } + + if (lookupLevel != searchTreeLevel) { + throw new IOException("Reached a data block at level " + lookupLevel + + " but the number of levels is " + searchTreeLevel); + } + + return block; + } + + /** + * An approximation to the {@link HFile}'s mid-key. Operates on block + * boundaries, and does not go inside blocks. In other words, returns the + * first key of the middle block of the file. + * + * @return the first key of the middle block + */ + public byte[] midkey() throws IOException { + if (rootCount == 0) + throw new IOException("HFile empty"); + + byte[] midKey = this.midKey.get(); + if (midKey != null) + return midKey; + + if (midLeafBlockOffset >= 0) { + if (blockReader == null) { + throw new IOException("Have to read the middle leaf block but " + + "no block reader available"); + } + HFileBlock midLeafBlock = blockReader.readBlockData( + midLeafBlockOffset, midLeafBlockOnDiskSize, -1, true); + ByteBuffer b = midLeafBlock.getBufferWithoutHeader(); + int numDataBlocks = b.getInt(); + int keyRelOffset = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 1)); + int keyLen = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 2)) - + keyRelOffset; + int keyOffset = b.arrayOffset() + + Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset + + SECONDARY_INDEX_ENTRY_OVERHEAD; + midKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset + keyLen); + } else { + // The middle of the root-level index. + midKey = blockKeys[(rootCount - 1) / 2]; + } + + this.midKey.set(midKey); + return midKey; + } + + /** + * @param i from 0 to {@link #getRootBlockCount() - 1} + */ + public byte[] getRootBlockKey(int i) { + return blockKeys[i]; + } + + /** + * @param i from 0 to {@link #getRootBlockCount() - 1} + */ + public long getRootBlockOffset(int i) { + return blockOffsets[i]; + } + + /** + * @param i zero-based index of a root-level block + * @return the on-disk size of the root-level block for version 2, or the + * uncompressed size for version 1 + */ + public int getRootBlockDataSize(int i) { + return blockDataSizes[i]; + } + + /** + * @return the number of root-level blocks in this block index + */ + public int getRootBlockCount() { + return rootCount; + } + + /** + * Finds the root-level index block containing the given key. + * + * @param key + * Key to find + * @return Offset of block containing key (between 0 and the + * number of blocks - 1) or -1 if this file does not contain the + * request. + */ + public int rootBlockContainingKey(final byte[] key, int offset, + int length) { + int pos = Bytes.binarySearch(blockKeys, key, offset, length, + comparator); + // pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see + // binarySearch's javadoc. + + if (pos >= 0) { + // This means this is an exact match with an element of blockKeys. + assert pos < blockKeys.length; + return pos; + } + + // Otherwise, pos = -(i + 1), where blockKeys[i - 1] < key < blockKeys[i], + // and i is in [0, blockKeys.length]. We are returning j = i - 1 such that + // blockKeys[j] <= key < blockKeys[j + 1]. In particular, j = -1 if + // key < blockKeys[0], meaning the file does not contain the given key. + + int i = -pos - 1; + assert 0 <= i && i <= blockKeys.length; + return i - 1; + } + + /** + * Adds a new entry in the root block index. Only used when reading. + * + * @param key Last key in the block + * @param offset file offset where the block is stored + * @param dataSize the uncompressed data size + */ + private void add(final byte[] key, final long offset, final int dataSize) { + blockOffsets[rootCount] = offset; + blockKeys[rootCount] = key; + blockDataSizes[rootCount] = dataSize; + + rootCount++; + rootByteSize += SECONDARY_INDEX_ENTRY_OVERHEAD + key.length; + } + + /** + * Performs a binary search over a non-root level index block. Utilizes the + * secondary index, which records the offsets of (offset, onDiskSize, + * firstKey) tuples of all entries. + * + * @param key the key we are searching for offsets to individual entries in + * the blockIndex buffer + * @param keyOffset the offset of the key in its byte array + * @param keyLength the length of the key + * @param nonRootIndex the non-root index block buffer, starting with the + * secondary index. The position is ignored. + * @return the index i in [0, numEntries - 1] such that keys[i] <= key < + * keys[i + 1], if keys is the array of all keys being searched, or + * -1 otherwise + * @throws IOException + */ + static int binarySearchNonRootIndex(byte[] key, int keyOffset, + int keyLength, ByteBuffer nonRootIndex, + RawComparator comparator) { + + int numEntries = nonRootIndex.getInt(0); + int low = 0; + int high = numEntries - 1; + int mid = 0; + + // Entries start after the number of entries and the secondary index. + // The secondary index takes numEntries + 1 ints. + int entriesOffset = Bytes.SIZEOF_INT * (numEntries + 2); + + // If we imagine that keys[-1] = -Infinity and + // keys[numEntries] = Infinity, then we are maintaining an invariant that + // keys[low - 1] < key < keys[high + 1] while narrowing down the range. + + while (low <= high) { + mid = (low + high) >>> 1; + + // Midkey's offset relative to the end of secondary index + int midKeyRelOffset = nonRootIndex.getInt( + Bytes.SIZEOF_INT * (mid + 1)); + + // The offset of the middle key in the blockIndex buffer + int midKeyOffset = entriesOffset // Skip secondary index + + midKeyRelOffset // Skip all entries until mid + + SECONDARY_INDEX_ENTRY_OVERHEAD; // Skip offset and on-disk-size + + // We subtract the two consecutive secondary index elements, which + // gives us the size of the whole (offset, onDiskSize, key) tuple. We + // then need to subtract the overhead of offset and onDiskSize. + int midLength = nonRootIndex.getInt(Bytes.SIZEOF_INT * (mid + 2)) - + midKeyRelOffset - SECONDARY_INDEX_ENTRY_OVERHEAD; + + // we have to compare in this order, because the comparator order + // has special logic when the 'left side' is a special key. + int cmp = comparator.compare(key, keyOffset, keyLength, + nonRootIndex.array(), nonRootIndex.arrayOffset() + midKeyOffset, + midLength); + + // key lives above the midpoint + if (cmp > 0) + low = mid + 1; // Maintain the invariant that keys[low - 1] < key + // key lives below the midpoint + else if (cmp < 0) + high = mid - 1; // Maintain the invariant that key < keys[high + 1] + else + return mid; // exact match + } + + // As per our invariant, keys[low - 1] < key < keys[high + 1], meaning + // that low - 1 < high + 1 and (low - high) <= 1. As per the loop break + // condition, low >= high + 1. Therefore, low = high + 1. + + if (low != high + 1) { + throw new IllegalStateException("Binary search broken: low=" + low + + " " + "instead of " + (high + 1)); + } + + // OK, our invariant says that keys[low - 1] < key < keys[low]. We need to + // return i such that keys[i] <= key < keys[i + 1]. Therefore i = low - 1. + int i = low - 1; + + // Some extra validation on the result. + if (i < -1 || i >= numEntries) { + throw new IllegalStateException("Binary search broken: result is " + + i + " but expected to be between -1 and (numEntries - 1) = " + + (numEntries - 1)); + } + + return i; + } + + /** + * Search for one key using the secondary index in a non-root block. In case + * of success, positions the provided buffer at the entry of interest, where + * the file offset and the on-disk-size can be read. + * + * @param nonRootBlock a non-root block without header. Initial position + * does not matter. + * @param key the byte array containing the key + * @param keyOffset the offset of the key in its byte array + * @param keyLength the length of the key + * @return true in the case the index entry containing the given key was + * found, false in the case the given key is before the first key + * + */ + static boolean locateNonRootIndexEntry(ByteBuffer nonRootBlock, byte[] key, + int keyOffset, int keyLength, RawComparator comparator) { + int entryIndex = binarySearchNonRootIndex(key, keyOffset, keyLength, + nonRootBlock, comparator); + + if (entryIndex == -1) { + return false; + } + + int numEntries = nonRootBlock.getInt(0); + + // The end of secondary index and the beginning of entries themselves. + int entriesOffset = Bytes.SIZEOF_INT * (numEntries + 2); + + // The offset of the entry we are interested in relative to the end of + // the secondary index. + int entryRelOffset = nonRootBlock.getInt(Bytes.SIZEOF_INT + * (1 + entryIndex)); + + nonRootBlock.position(entriesOffset + entryRelOffset); + return true; + } + + /** + * Read in the root-level index from the given input stream. Must match + * what was written into the root level by + * {@link BlockIndexWriter#writeIndexBlocks(FSDataOutputStream)} at the + * offset that function returned. + * + * @param in the buffered input stream or wrapped byte input stream + * @param numEntries the number of root-level index entries + * @throws IOException + */ + public void readRootIndex(DataInput in, final int numEntries) + throws IOException { + blockOffsets = new long[numEntries]; + blockKeys = new byte[numEntries][]; + blockDataSizes = new int[numEntries]; + + // If index size is zero, no index was written. + if (numEntries > 0) { + for (int i = 0; i < numEntries; ++i) { + long offset = in.readLong(); + int dataSize = in.readInt(); + byte[] key = Bytes.readByteArray(in); + add(key, offset, dataSize); + } + } + } + + /** + * Read the root-level metadata of a multi-level block index. Based on + * {@link #readRootIndex(DataInput, int)}, but also reads metadata + * necessary to compute the mid-key in a multi-level index. + * + * @param in the buffered or byte input stream to read from + * @param numEntries the number of root-level index entries + * @throws IOException + */ + public void readMultiLevelIndexRoot(DataInputStream in, + final int numEntries) throws IOException { + readRootIndex(in, numEntries); + if (in.available() < MID_KEY_METADATA_SIZE) { + // No mid-key metadata available. + return; + } + + midLeafBlockOffset = in.readLong(); + midLeafBlockOnDiskSize = in.readInt(); + midKeyEntry = in.readInt(); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("size=" + rootCount).append("\n"); + for (int i = 0; i < rootCount; i++) { + sb.append("key=").append(KeyValue.keyToString(blockKeys[i])) + .append("\n offset=").append(blockOffsets[i]) + .append(", dataSize=" + blockDataSizes[i]).append("\n"); + } + return sb.toString(); + } + + @Override + public long heapSize() { + long heapSize = ClassSize.align(6 * ClassSize.REFERENCE + + 3 * Bytes.SIZEOF_INT + ClassSize.OBJECT); + + // Mid-key metadata. + heapSize += MID_KEY_METADATA_SIZE; + + // Calculating the size of blockKeys + if (blockKeys != null) { + // Adding array + references overhead + heapSize += ClassSize.align(ClassSize.ARRAY + blockKeys.length + * ClassSize.REFERENCE); + + // Adding bytes + for (byte[] key : blockKeys) { + heapSize += ClassSize.align(ClassSize.ARRAY + key.length); + } + } + + if (blockOffsets != null) { + heapSize += ClassSize.align(ClassSize.ARRAY + blockOffsets.length + * Bytes.SIZEOF_LONG); + } + + if (blockDataSizes != null) { + heapSize += ClassSize.align(ClassSize.ARRAY + blockDataSizes.length + * Bytes.SIZEOF_INT); + } + + return ClassSize.align(heapSize); + } + + } + + /** + * Writes the block index into the output stream. Generate the tree from + * bottom up. The leaf level is written to disk as a sequence of inline + * blocks, if it is larger than a certain number of bytes. If the leaf level + * is not large enough, we write all entries to the root level instead. + * + * After all leaf blocks have been written, we end up with an index + * referencing the resulting leaf index blocks. If that index is larger than + * the allowed root index size, the writer will break it up into + * reasonable-size intermediate-level index block chunks write those chunks + * out, and create another index referencing those chunks. This will be + * repeated until the remaining index is small enough to become the root + * index. However, in most practical cases we will only have leaf-level + * blocks and the root index, or just the root index. + */ + public static class BlockIndexWriter implements InlineBlockWriter { + /** + * While the index is being written, this represents the current block + * index referencing all leaf blocks, with one exception. If the file is + * being closed and there are not enough blocks to complete even a single + * leaf block, no leaf blocks get written and this contains the entire + * block index. After all levels of the index were written by + * {@link #writeIndexBlocks(FSDataOutputStream)}, this contains the final + * root-level index. + */ + private BlockIndexChunk rootChunk = new BlockIndexChunk(); + + /** + * Current leaf-level chunk. New entries referencing data blocks get added + * to this chunk until it grows large enough to be written to disk. + */ + private BlockIndexChunk curInlineChunk = new BlockIndexChunk(); + + /** + * The number of block index levels. This is one if there is only root + * level (even empty), two if there a leaf level and root level, and is + * higher if there are intermediate levels. This is only final after + * {@link #writeIndexBlocks(FSDataOutputStream)} has been called. The + * initial value accounts for the root level, and will be increased to two + * as soon as we find out there is a leaf-level in + * {@link #blockWritten(long, int)}. + */ + private int numLevels = 1; + + private HFileBlock.Writer blockWriter; + private byte[] firstKey = null; + + /** + * The total number of leaf-level entries, i.e. entries referenced by + * leaf-level blocks. For the data block index this is equal to the number + * of data blocks. + */ + private long totalNumEntries; + + /** Total compressed size of all index blocks. */ + private long totalBlockOnDiskSize; + + /** Total uncompressed size of all index blocks. */ + private long totalBlockUncompressedSize; + + /** The maximum size guideline of all multi-level index blocks. */ + private int maxChunkSize; + + /** Whether we require this block index to always be single-level. */ + private boolean singleLevelOnly; + + /** Block cache, or null if cache-on-write is disabled */ + private BlockCache blockCache; + + /** Name to use for computing cache keys */ + private String nameForCaching; + + /** Creates a single-level block index writer */ + public BlockIndexWriter() { + this(null, null, null); + singleLevelOnly = true; + } + + /** + * Creates a multi-level block index writer. + * + * @param blockWriter the block writer to use to write index blocks + * @param blockCache if this is not null, index blocks will be cached + * on write into this block cache. + */ + public BlockIndexWriter(HFileBlock.Writer blockWriter, + BlockCache blockCache, String nameForCaching) { + if ((blockCache == null) != (nameForCaching == null)) { + throw new IllegalArgumentException("Block cache and file name for " + + "caching must be both specified or both null"); + } + + this.blockWriter = blockWriter; + this.blockCache = blockCache; + this.nameForCaching = nameForCaching; + this.maxChunkSize = HFileBlockIndex.DEFAULT_MAX_CHUNK_SIZE; + } + + public void setMaxChunkSize(int maxChunkSize) { + if (maxChunkSize <= 0) { + throw new IllegalArgumentException("Invald maximum index block size"); + } + this.maxChunkSize = maxChunkSize; + } + + /** + * Writes the root level and intermediate levels of the block index into + * the output stream, generating the tree from bottom up. Assumes that the + * leaf level has been inline-written to the disk if there is enough data + * for more than one leaf block. We iterate by breaking the current level + * of the block index, starting with the index of all leaf-level blocks, + * into chunks small enough to be written to disk, and generate its parent + * level, until we end up with a level small enough to become the root + * level. + * + * If the leaf level is not large enough, there is no inline block index + * anymore, so we only write that level of block index to disk as the root + * level. + * + * @param out FSDataOutputStream + * @return position at which we entered the root-level index. + * @throws IOException + */ + public long writeIndexBlocks(FSDataOutputStream out) throws IOException { + if (curInlineChunk.getNumEntries() != 0) { + throw new IOException("Trying to write a multi-level block index, " + + "but are " + curInlineChunk.getNumEntries() + " entries in the " + + "last inline chunk."); + } + + // We need to get mid-key metadata before we create intermediate + // indexes and overwrite the root chunk. + byte[] midKeyMetadata = numLevels > 1 ? rootChunk.getMidKeyMetadata() + : null; + + while (rootChunk.getRootSize() > maxChunkSize) { + rootChunk = writeIntermediateLevel(out, rootChunk); + numLevels += 1; + } + + // write the root level + long rootLevelIndexPos = out.getPos(); + + { + DataOutput blockStream = blockWriter.startWriting(BlockType.ROOT_INDEX, + false); + rootChunk.writeRoot(blockStream); + if (midKeyMetadata != null) + blockStream.write(midKeyMetadata); + blockWriter.writeHeaderAndData(out); + } + + // Add root index block size + totalBlockOnDiskSize += blockWriter.getOnDiskSizeWithoutHeader(); + totalBlockUncompressedSize += + blockWriter.getUncompressedSizeWithoutHeader(); + + LOG.info("Wrote a " + numLevels + "-level index with root level at pos " + + out.getPos() + ", " + rootChunk.getNumEntries() + + " root-level entries, " + totalNumEntries + " total entries, " + + totalBlockOnDiskSize + " bytes total on-disk size, " + + totalBlockUncompressedSize + " bytes total uncompressed size."); + + return rootLevelIndexPos; + } + + /** + * Writes the block index data as a single level only. Does not do any + * block framing. + * + * @param out the buffered output stream to write the index to. Typically a + * stream writing into an {@link HFile} block. + * @param description a short description of the index being written. Used + * in a log message. + * @throws IOException + */ + public void writeSingleLevelIndex(DataOutput out, String description) + throws IOException { + expectNumLevels(1); + + if (!singleLevelOnly) + throw new IOException("Single-level mode is turned off"); + + if (rootChunk.getNumEntries() > 0) + throw new IOException("Root-level entries already added in " + + "single-level mode"); + + rootChunk = curInlineChunk; + curInlineChunk = new BlockIndexChunk(); + + LOG.info("Wrote a single-level " + description + " index with " + + rootChunk.getNumEntries() + " entries, " + rootChunk.getRootSize() + + " bytes"); + rootChunk.writeRoot(out); + } + + /** + * Split the current level of the block index into intermediate index + * blocks of permitted size and write those blocks to disk. Return the next + * level of the block index referencing those intermediate-level blocks. + * + * @param out + * @param currentLevel the current level of the block index, such as the a + * chunk referencing all leaf-level index blocks + * @return the parent level block index, which becomes the root index after + * a few (usually zero) iterations + * @throws IOException + */ + private BlockIndexChunk writeIntermediateLevel(FSDataOutputStream out, + BlockIndexChunk currentLevel) throws IOException { + // Entries referencing intermediate-level blocks we are about to create. + BlockIndexChunk parent = new BlockIndexChunk(); + + // The current intermediate-level block index chunk. + BlockIndexChunk curChunk = new BlockIndexChunk(); + + for (int i = 0; i < currentLevel.getNumEntries(); ++i) { + curChunk.add(currentLevel.getBlockKey(i), + currentLevel.getBlockOffset(i), currentLevel.getOnDiskDataSize(i)); + + if (curChunk.getRootSize() >= maxChunkSize) + writeIntermediateBlock(out, parent, curChunk); + } + + if (curChunk.getNumEntries() > 0) { + writeIntermediateBlock(out, parent, curChunk); + } + + return parent; + } + + private void writeIntermediateBlock(FSDataOutputStream out, + BlockIndexChunk parent, BlockIndexChunk curChunk) throws IOException { + long beginOffset = out.getPos(); + DataOutputStream dos = blockWriter.startWriting( + BlockType.INTERMEDIATE_INDEX, cacheOnWrite()); + curChunk.writeNonRoot(dos); + byte[] curFirstKey = curChunk.getBlockKey(0); + blockWriter.writeHeaderAndData(out); + + if (blockCache != null) { + blockCache.cacheBlock(HFile.getBlockCacheKey(nameForCaching, + beginOffset), blockWriter.getBlockForCaching()); + } + + // Add intermediate index block size + totalBlockOnDiskSize += blockWriter.getOnDiskSizeWithoutHeader(); + totalBlockUncompressedSize += + blockWriter.getUncompressedSizeWithoutHeader(); + + // OFFSET is the beginning offset the chunk of block index entries. + // SIZE is the total byte size of the chunk of block index entries + // + the secondary index size + // FIRST_KEY is the first key in the chunk of block index + // entries. + parent.add(curFirstKey, beginOffset, + blockWriter.getOnDiskSizeWithHeader()); + + // clear current block index chunk + curChunk.clear(); + curFirstKey = null; + } + + /** + * @return how many block index entries there are in the root level + */ + public final int getNumRootEntries() { + return rootChunk.getNumEntries(); + } + + /** + * @return the number of levels in this block index. + */ + public int getNumLevels() { + return numLevels; + } + + private void expectNumLevels(int expectedNumLevels) { + if (numLevels != expectedNumLevels) { + throw new IllegalStateException("Number of block index levels is " + + numLevels + "but is expected to be " + expectedNumLevels); + } + } + + /** + * Whether there is an inline block ready to be written. In general, we + * write an leaf-level index block as an inline block as soon as its size + * as serialized in the non-root format reaches a certain threshold. + */ + @Override + public boolean shouldWriteBlock(boolean closing) { + if (singleLevelOnly) + throw new UnsupportedOperationException(INLINE_BLOCKS_NOT_ALLOWED); + + if (curInlineChunk.getNumEntries() == 0) + return false; + + // We do have some entries in the current inline chunk. + if (closing) { + if (rootChunk.getNumEntries() == 0) { + // We did not add any leaf-level blocks yet. Instead of creating a + // leaf level with one block, move these entries to the root level. + + expectNumLevels(1); + rootChunk = curInlineChunk; + curInlineChunk = new BlockIndexChunk(); + return false; + } + + return true; + } else { + return curInlineChunk.getNonRootSize() >= maxChunkSize; + } + } + + /** + * Write out the current inline index block. Inline blocks are non-root + * blocks, so the non-root index format is used. + * + * @param out + * @param position The beginning offset of the inline block in the file not + * include the header. + */ + @Override + public void writeInlineBlock(DataOutput out) throws IOException { + if (singleLevelOnly) + throw new UnsupportedOperationException(INLINE_BLOCKS_NOT_ALLOWED); + + // Write the inline block index to the output stream in the non-root + // index block format. + curInlineChunk.writeNonRoot(out); + + // Save the first key of the inline block so that we can add it to the + // parent-level index. + firstKey = curInlineChunk.getBlockKey(0); + + // Start a new inline index block + curInlineChunk.clear(); + } + + /** + * Called after an inline block has been written so that we can add an + * entry referring to that block to the parent-level index. + */ + @Override + public void blockWritten(long offset, int onDiskSize, int uncompressedSize) + { + // Add leaf index block size + totalBlockOnDiskSize += onDiskSize; + totalBlockUncompressedSize += uncompressedSize; + + if (singleLevelOnly) + throw new UnsupportedOperationException(INLINE_BLOCKS_NOT_ALLOWED); + + if (firstKey == null) { + throw new IllegalStateException("Trying to add second-level index " + + "entry with offset=" + offset + " and onDiskSize=" + onDiskSize + + "but the first key was not set in writeInlineBlock"); + } + + if (rootChunk.getNumEntries() == 0) { + // We are writing the first leaf block, so increase index level. + expectNumLevels(1); + numLevels = 2; + } + + // Add another entry to the second-level index. Include the number of + // entries in all previous leaf-level chunks for mid-key calculation. + rootChunk.add(firstKey, offset, onDiskSize, totalNumEntries); + firstKey = null; + } + + @Override + public BlockType getInlineBlockType() { + return BlockType.LEAF_INDEX; + } + + /** + * Add one index entry to the current leaf-level block. When the leaf-level + * block gets large enough, it will be flushed to disk as an inline block. + * + * @param firstKey the first key of the data block + * @param blockOffset the offset of the data block + * @param blockDataSize the on-disk size of the data block ({@link HFile} + * format version 2), or the uncompressed size of the data block ( + * {@link HFile} format version 1). + */ + public void addEntry(byte[] firstKey, long blockOffset, int blockDataSize) + { + curInlineChunk.add(firstKey, blockOffset, blockDataSize); + ++totalNumEntries; + } + + /** + * @throws IOException if we happened to write a multi-level index. + */ + public void ensureSingleLevel() throws IOException { + if (numLevels > 1) { + throw new IOException ("Wrote a " + numLevels + "-level index with " + + rootChunk.getNumEntries() + " root-level entries, but " + + "this is expected to be a single-level block index."); + } + } + + /** + * @return true if we are using cache-on-write. This is configured by the + * caller of the constructor by either passing a valid block cache + * or null. + */ + @Override + public boolean cacheOnWrite() { + return blockCache != null; + } + + /** + * The total uncompressed size of the root index block, intermediate-level + * index blocks, and leaf-level index blocks. + * + * @return the total uncompressed size of all index blocks + */ + public long getTotalUncompressedSize() { + return totalBlockUncompressedSize; + } + + } + + /** + * A single chunk of the block index in the process of writing. The data in + * this chunk can become a leaf-level, intermediate-level, or root index + * block. + */ + static class BlockIndexChunk { + + /** First keys of the key range corresponding to each index entry. */ + private final List blockKeys = new ArrayList(); + + /** Block offset in backing stream. */ + private final List blockOffsets = new ArrayList(); + + /** On-disk data sizes of lower-level data or index blocks. */ + private final List onDiskDataSizes = new ArrayList(); + + /** + * The cumulative number of sub-entries, i.e. entries on deeper-level block + * index entries. numSubEntriesAt[i] is the number of sub-entries in the + * blocks corresponding to this chunk's entries #0 through #i inclusively. + */ + private final List numSubEntriesAt = new ArrayList(); + + /** + * The offset of the next entry to be added, relative to the end of the + * "secondary index" in the "non-root" format representation of this index + * chunk. This is the next value to be added to the secondary index. + */ + private int curTotalNonRootEntrySize = 0; + + /** + * The accumulated size of this chunk if stored in the root index format. + */ + private int curTotalRootSize = 0; + + /** + * The "secondary index" used for binary search over variable-length + * records in a "non-root" format block. These offsets are relative to the + * end of this secondary index. + */ + private final List secondaryIndexOffsetMarks = + new ArrayList(); + + /** + * Adds a new entry to this block index chunk. + * + * @param firstKey the first key in the block pointed to by this entry + * @param blockOffset the offset of the next-level block pointed to by this + * entry + * @param onDiskDataSize the on-disk data of the block pointed to by this + * entry, including header size + * @param curTotalNumSubEntries if this chunk is the root index chunk under + * construction, this specifies the current total number of + * sub-entries in all leaf-level chunks, including the one + * corresponding to the second-level entry being added. + */ + void add(byte[] firstKey, long blockOffset, int onDiskDataSize, + long curTotalNumSubEntries) { + // Record the offset for the secondary index + secondaryIndexOffsetMarks.add(curTotalNonRootEntrySize); + curTotalNonRootEntrySize += SECONDARY_INDEX_ENTRY_OVERHEAD + + firstKey.length; + + curTotalRootSize += Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT + + WritableUtils.getVIntSize(firstKey.length) + firstKey.length; + + blockKeys.add(firstKey); + blockOffsets.add(blockOffset); + onDiskDataSizes.add(onDiskDataSize); + + if (curTotalNumSubEntries != -1) { + numSubEntriesAt.add(curTotalNumSubEntries); + + // Make sure the parallel arrays are in sync. + if (numSubEntriesAt.size() != blockKeys.size()) { + throw new IllegalStateException("Only have key/value count " + + "stats for " + numSubEntriesAt.size() + " block index " + + "entries out of " + blockKeys.size()); + } + } + } + + /** + * The same as {@link #add(byte[], long, int, long)} but does not take the + * key/value into account. Used for single-level indexes. + * + * @see {@link #add(byte[], long, int, long)} + */ + public void add(byte[] firstKey, long blockOffset, int onDiskDataSize) { + add(firstKey, blockOffset, onDiskDataSize, -1); + } + + public void clear() { + blockKeys.clear(); + blockOffsets.clear(); + onDiskDataSizes.clear(); + secondaryIndexOffsetMarks.clear(); + numSubEntriesAt.clear(); + curTotalNonRootEntrySize = 0; + curTotalRootSize = 0; + } + + /** + * Finds the entry corresponding to the deeper-level index block containing + * the given deeper-level entry (a "sub-entry"), assuming a global 0-based + * ordering of sub-entries. + * + *

+ * Implementation note. We are looking for i such that + * numSubEntriesAt[i - 1] <= k < numSubEntriesAt[i], because a deeper-level + * block #i (0-based) contains sub-entries # numSubEntriesAt[i - 1]'th + * through numSubEntriesAt[i] - 1, assuming a global 0-based ordering of + * sub-entries. i is by definition the insertion point of k in + * numSubEntriesAt. + * + * @param k sub-entry index, from 0 to the total number sub-entries - 1 + * @return the 0-based index of the entry corresponding to the given + * sub-entry + */ + public int getEntryBySubEntry(long k) { + // We define mid-key as the key corresponding to k'th sub-entry + // (0-based). + + int i = Collections.binarySearch(numSubEntriesAt, k); + + // Exact match: cumulativeWeight[i] = k. This means chunks #0 through + // #i contain exactly k sub-entries, and the sub-entry #k (0-based) + // is in the (i + 1)'th chunk. + if (i >= 0) + return i + 1; + + // Inexact match. Return the insertion point. + return -i - 1; + } + + /** + * Used when writing the root block index of a multi-level block index. + * Serializes additional information allowing to efficiently identify the + * mid-key. + * + * @return a few serialized fields for finding the mid-key + * @throws IOException if could not create metadata for computing mid-key + */ + public byte[] getMidKeyMetadata() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream( + MID_KEY_METADATA_SIZE); + DataOutputStream baosDos = new DataOutputStream(baos); + long totalNumSubEntries = numSubEntriesAt.get(blockKeys.size() - 1); + if (totalNumSubEntries == 0) { + throw new IOException("No leaf-level entries, mid-key unavailable"); + } + long midKeySubEntry = (totalNumSubEntries - 1) / 2; + int midKeyEntry = getEntryBySubEntry(midKeySubEntry); + + baosDos.writeLong(blockOffsets.get(midKeyEntry)); + baosDos.writeInt(onDiskDataSizes.get(midKeyEntry)); + + long numSubEntriesBefore = midKeyEntry > 0 + ? numSubEntriesAt.get(midKeyEntry - 1) : 0; + long subEntryWithinEntry = midKeySubEntry - numSubEntriesBefore; + if (subEntryWithinEntry < 0 || subEntryWithinEntry > Integer.MAX_VALUE) + { + throw new IOException("Could not identify mid-key index within the " + + "leaf-level block containing mid-key: out of range (" + + subEntryWithinEntry + ", numSubEntriesBefore=" + + numSubEntriesBefore + ", midKeySubEntry=" + midKeySubEntry + + ")"); + } + + baosDos.writeInt((int) subEntryWithinEntry); + + if (baosDos.size() != MID_KEY_METADATA_SIZE) { + throw new IOException("Could not write mid-key metadata: size=" + + baosDos.size() + ", correct size: " + MID_KEY_METADATA_SIZE); + } + + // Close just to be good citizens, although this has no effect. + baos.close(); + + return baos.toByteArray(); + } + + /** + * Writes the block index chunk in the non-root index block format. This + * format contains the number of entries, an index of integer offsets + * for quick binary search on variable-length records, and tuples of + * block offset, on-disk block size, and the first key for each entry. + * + * @param out + * @throws IOException + */ + void writeNonRoot(DataOutput out) throws IOException { + // The number of entries in the block. + out.writeInt(blockKeys.size()); + + if (secondaryIndexOffsetMarks.size() != blockKeys.size()) { + throw new IOException("Corrupted block index chunk writer: " + + blockKeys.size() + " entries but " + + secondaryIndexOffsetMarks.size() + " secondary index items"); + } + + // For each entry, write a "secondary index" of relative offsets to the + // entries from the end of the secondary index. This works, because at + // read time we read the number of entries and know where the secondary + // index ends. + for (int currentSecondaryIndex : secondaryIndexOffsetMarks) + out.writeInt(currentSecondaryIndex); + + // We include one other element in the secondary index to calculate the + // size of each entry more easily by subtracting secondary index elements. + out.writeInt(curTotalNonRootEntrySize); + + for (int i = 0; i < blockKeys.size(); ++i) { + out.writeLong(blockOffsets.get(i)); + out.writeInt(onDiskDataSizes.get(i)); + out.write(blockKeys.get(i)); + } + } + + /** + * @return the size of this chunk if stored in the non-root index block + * format + */ + int getNonRootSize() { + return Bytes.SIZEOF_INT // Number of entries + + Bytes.SIZEOF_INT * (blockKeys.size() + 1) // Secondary index + + curTotalNonRootEntrySize; // All entries + } + + /** + * Writes this chunk into the given output stream in the root block index + * format. This format is similar to the {@link HFile} version 1 block + * index format, except that we store on-disk size of the block instead of + * its uncompressed size. + * + * @param out the data output stream to write the block index to. Typically + * a stream writing into an {@link HFile} block. + * @throws IOException + */ + void writeRoot(DataOutput out) throws IOException { + for (int i = 0; i < blockKeys.size(); ++i) { + out.writeLong(blockOffsets.get(i)); + out.writeInt(onDiskDataSizes.get(i)); + Bytes.writeByteArray(out, blockKeys.get(i)); + } + } + + /** + * @return the size of this chunk if stored in the root index block format + */ + int getRootSize() { + return curTotalRootSize; + } + + /** + * @return the number of entries in this block index chunk + */ + public int getNumEntries() { + return blockKeys.size(); + } + + public byte[] getBlockKey(int i) { + return blockKeys.get(i); + } + + public long getBlockOffset(int i) { + return blockOffsets.get(i); + } + + public int getOnDiskDataSize(int i) { + return onDiskDataSizes.get(i); + } + + public long getCumulativeNumKV(int i) { + if (i < 0) + return 0; + return numSubEntriesAt.get(i); + } + + } + + /** + * @return true if the given configuration specifies that we should + * cache-on-write index blocks + */ + public static boolean shouldCacheOnWrite(Configuration conf) { + return conf.getBoolean(CACHE_INDEX_BLOCKS_ON_WRITE_KEY, false); + } + + public static int getMaxChunkSize(Configuration conf) { + return conf.getInt(MAX_CHUNK_SIZE_KEY, DEFAULT_MAX_CHUNK_SIZE); + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java new file mode 100644 index 0000000..e9ee0a7 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -0,0 +1,308 @@ + +/* + * Copyright 2011 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.DataInput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.cli.PosixParser; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; +import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; +import org.apache.hadoop.hbase.util.BloomFilter; +import org.apache.hadoop.hbase.util.BloomFilterFactory; +import org.apache.hadoop.hbase.util.ByteBloomFilter; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.Writables; + +/** + * Implements pretty-printing functionality for {@link HFile}s. + */ +public class HFilePrettyPrinter { + + private static final Log LOG = LogFactory.getLog(HFilePrettyPrinter.class); + + private Options options = new Options(); + + private boolean verbose; + private boolean printValue; + private boolean printKey; + private boolean shouldPrintMeta; + private boolean printBlocks; + private boolean checkRow; + private boolean checkFamily; + + private Configuration conf; + + private List files = new ArrayList(); + private int count; + + private static final String FOUR_SPACES = " "; + + public HFilePrettyPrinter() { + options.addOption("v", "verbose", false, + "Verbose output; emits file and meta data delimiters"); + options.addOption("p", "printkv", false, "Print key/value pairs"); + options.addOption("e", "printkey", false, "Print keys"); + options.addOption("m", "printmeta", false, "Print meta data of file"); + options.addOption("b", "printblocks", false, "Print block index meta data"); + options.addOption("k", "checkrow", false, + "Enable row order check; looks for out-of-order keys"); + options.addOption("a", "checkfamily", false, "Enable family check"); + options.addOption("f", "file", true, + "File to scan. Pass full-path; e.g. hdfs://a:9000/hbase/.META./12/34"); + options.addOption("r", "region", true, + "Region to scan. Pass region name; e.g. '.META.,,1'"); + } + + public boolean parseOptions(String args[]) throws ParseException, + IOException { + if (args.length == 0) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp("HFile", options, true); + return false; + } + CommandLineParser parser = new PosixParser(); + CommandLine cmd = parser.parse(options, args); + + verbose = cmd.hasOption("v"); + printValue = cmd.hasOption("p"); + printKey = cmd.hasOption("e") || printValue; + shouldPrintMeta = cmd.hasOption("m"); + printBlocks = cmd.hasOption("b"); + checkRow = cmd.hasOption("k"); + checkFamily = cmd.hasOption("a"); + + if (cmd.hasOption("f")) { + files.add(new Path(cmd.getOptionValue("f"))); + } + + if (cmd.hasOption("r")) { + String regionName = cmd.getOptionValue("r"); + byte[] rn = Bytes.toBytes(regionName); + byte[][] hri = HRegionInfo.parseRegionName(rn); + Path rootDir = FSUtils.getRootDir(conf); + Path tableDir = new Path(rootDir, Bytes.toString(hri[0])); + String enc = HRegionInfo.encodeRegionName(rn); + Path regionDir = new Path(tableDir, enc); + if (verbose) + System.out.println("region dir -> " + regionDir); + List regionFiles = HFile.getStoreFiles(FileSystem.get(conf), + regionDir); + if (verbose) + System.out.println("Number of region files found -> " + + regionFiles.size()); + if (verbose) { + int i = 1; + for (Path p : regionFiles) { + if (verbose) + System.out.println("Found file[" + i++ + "] -> " + p); + } + } + files.addAll(regionFiles); + } + + return true; + } + + /** + * Runs the command-line pretty-printer, and returns the desired command + * exit code (zero for success, non-zero for failure). + */ + public int run(String[] args) { + conf = HBaseConfiguration.create(); + conf.set("fs.defaultFS", + conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR)); + conf.set("fs.default.name", + conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR)); + try { + if (!parseOptions(args)) + return 1; + } catch (IOException ex) { + LOG.error("Error parsing command-line options", ex); + return 1; + } catch (ParseException ex) { + LOG.error("Error parsing command-line options", ex); + return 1; + } + + // iterate over all files found + for (Path fileName : files) { + try { + processFile(fileName); + } catch (IOException ex) { + LOG.error("Error reading " + fileName, ex); + } + } + + if (verbose || printKey) { + System.out.println("Scanned kv count -> " + count); + } + + return 0; + } + + private void processFile(Path file) throws IOException { + if (verbose) + System.out.println("Scanning -> " + file); + FileSystem fs = file.getFileSystem(conf); + if (!fs.exists(file)) { + System.err.println("ERROR, file doesnt exist: " + file); + } + + HFile.Reader reader = HFile.createReader(fs, file, null, false, false); + + Map fileInfo = reader.loadFileInfo(); + + if (verbose || printKey || checkRow || checkFamily) { + + // scan over file and read key/value's and check if requested + HFileScanner scanner = reader.getScanner(false, false, false); + scanner.seekTo(); + scanKeysValues(file, count, scanner); + } + + // print meta data + if (shouldPrintMeta) { + printMeta(reader, fileInfo); + } + + if (printBlocks) { + System.out.println("Block Index:"); + System.out.println(reader.getDataBlockIndexReader()); + } + + reader.close(); + } + + private void scanKeysValues(Path file, int count, HFileScanner scanner) + throws IOException { + KeyValue pkv = null; + do { + KeyValue kv = scanner.getKeyValue(); + // dump key value + if (printKey) { + System.out.print("K: " + kv); + if (printValue) { + System.out.print(" V: " + Bytes.toStringBinary(kv.getValue())); + } + System.out.println(); + } + // check if rows are in order + if (checkRow && pkv != null) { + if (Bytes.compareTo(pkv.getRow(), kv.getRow()) > 0) { + System.err.println("WARNING, previous row is greater then" + + " current row\n\tfilename -> " + file + "\n\tprevious -> " + + Bytes.toStringBinary(pkv.getKey()) + "\n\tcurrent -> " + + Bytes.toStringBinary(kv.getKey())); + } + } + // check if families are consistent + if (checkFamily) { + String fam = Bytes.toString(kv.getFamily()); + if (!file.toString().contains(fam)) { + System.err.println("WARNING, filename does not match kv family," + + "\n\tfilename -> " + file + "\n\tkeyvalue -> " + + Bytes.toStringBinary(kv.getKey())); + } + if (pkv != null + && !Bytes.equals(pkv.getFamily(), kv.getFamily())) { + System.err.println("WARNING, previous kv has different family" + + " compared to current key\n\tfilename -> " + file + + "\n\tprevious -> " + Bytes.toStringBinary(pkv.getKey()) + + "\n\tcurrent -> " + Bytes.toStringBinary(kv.getKey())); + } + } + pkv = kv; + ++count; + } while (scanner.next()); + } + + /** + * Format a string of the form "k1=v1, k2=v2, ..." into separate lines + * with a four-space indentation. + */ + private static String asSeparateLines(String keyValueStr) { + return keyValueStr.replaceAll(", ([a-zA-Z]+=)", + ",\n" + FOUR_SPACES + "$1"); + } + + private void printMeta(HFile.Reader reader, Map fileInfo) + throws IOException { + System.out.println("Block index size as per heapsize: " + + reader.indexSize()); + System.out.println(asSeparateLines(reader.toString())); + System.out.println("Trailer:\n " + + asSeparateLines(reader.getTrailer().toString())); + System.out.println("Fileinfo:"); + for (Map.Entry e : fileInfo.entrySet()) { + System.out.print(FOUR_SPACES + Bytes.toString(e.getKey()) + " = "); + if (Bytes.compareTo(e.getKey(), Bytes.toBytes("MAX_SEQ_ID_KEY")) == 0) { + long seqid = Bytes.toLong(e.getValue()); + System.out.println(seqid); + } else if (Bytes.compareTo(e.getKey(), Bytes.toBytes("TIMERANGE")) == 0) { + TimeRangeTracker timeRangeTracker = new TimeRangeTracker(); + Writables.copyWritable(e.getValue(), timeRangeTracker); + System.out.println(timeRangeTracker.getMinimumTimestamp() + "...." + + timeRangeTracker.getMaximumTimestamp()); + } else if (Bytes.compareTo(e.getKey(), FileInfo.AVG_KEY_LEN) == 0 + || Bytes.compareTo(e.getKey(), FileInfo.AVG_VALUE_LEN) == 0) { + System.out.println(Bytes.toInt(e.getValue())); + } else { + System.out.println(Bytes.toStringBinary(e.getValue())); + } + } + + System.out.println("Mid-key: " + Bytes.toStringBinary(reader.midkey())); + + // Printing bloom information + DataInput bloomMeta = reader.getBloomFilterMetadata(); + BloomFilter bloomFilter = null; + if (bloomMeta != null) + bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader); + + System.out.println("Bloom filter:"); + if (bloomFilter != null) { + System.out.println(FOUR_SPACES + bloomFilter.toString().replaceAll( + ByteBloomFilter.STATS_RECORD_SEP, "\n" + FOUR_SPACES)); + } else { + System.out.println(FOUR_SPACES + "Not present"); + } + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java new file mode 100644 index 0000000..13d5e70 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java @@ -0,0 +1,666 @@ +/* + * Copyright 2011 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.ByteArrayInputStream; +import java.io.DataInput; +import java.io.DataInputStream; +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.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; +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.IOUtils; +import org.apache.hadoop.io.RawComparator; + +import com.google.common.base.Preconditions; + +/** + * {@link HFile} reader for version 1. + */ +public class HFileReaderV1 extends AbstractHFileReader { + private static final Log LOG = LogFactory.getLog(HFileReaderV1.class); + + private volatile boolean fileInfoLoaded = false; + + /** + * Opens a HFile. You must load the index before you can + * use it by calling {@link #loadFileInfo()}. + * + * @param fsdis input stream. Caller is responsible for closing the passed + * stream. + * @param size Length of the stream. + * @param blockCache block cache. Pass null if none. + * @param inMemory whether blocks should be marked as in-memory in cache + * @param evictOnClose whether blocks in cache should be evicted on close + * @throws IOException + */ + public HFileReaderV1(Path path, FixedFileTrailer trailer, + final FSDataInputStream fsdis, final long size, + final boolean closeIStream, + final BlockCache blockCache, final boolean inMemory, + final boolean evictOnClose) { + super(path, trailer, fsdis, size, closeIStream, blockCache, inMemory, + evictOnClose); + + trailer.expectVersion(1); + fsBlockReader = new HFileBlock.FSReaderV1(fsdis, compressAlgo, fileSize); + } + + private byte[] readAllIndex(final FSDataInputStream in, + final long indexOffset, final int indexSize) throws IOException { + byte[] allIndex = new byte[indexSize]; + in.seek(indexOffset); + IOUtils.readFully(in, allIndex, 0, allIndex.length); + + return allIndex; + } + + /** + * Read in the index and file info. + * + * @return A map of fileinfo data. + * @see {@link Writer#appendFileInfo(byte[], byte[])}. + * @throws IOException + */ + @Override + public FileInfo loadFileInfo() throws IOException { + if (fileInfoLoaded) + return fileInfo; + + // Read in the fileinfo and get what we need from it. + istream.seek(trailer.getFileInfoOffset()); + fileInfo = new FileInfo(); + fileInfo.readFields(istream); + lastKey = fileInfo.get(FileInfo.LASTKEY); + avgKeyLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_KEY_LEN)); + avgValueLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_VALUE_LEN)); + + // Comparator is stored in the file info in version 1. + String clazzName = Bytes.toString(fileInfo.get(FileInfo.COMPARATOR)); + comparator = getComparator(clazzName); + + dataBlockIndexReader = + new HFileBlockIndex.BlockIndexReader(comparator, 1); + metaBlockIndexReader = + new HFileBlockIndex.BlockIndexReader(Bytes.BYTES_RAWCOMPARATOR, 1); + + int sizeToLoadOnOpen = (int) (fileSize - trailer.getLoadOnOpenDataOffset() - + trailer.getTrailerSize()); + byte[] dataAndMetaIndex = readAllIndex(istream, + trailer.getLoadOnOpenDataOffset(), sizeToLoadOnOpen); + + ByteArrayInputStream bis = new ByteArrayInputStream(dataAndMetaIndex); + DataInputStream dis = new DataInputStream(bis); + + // Read in the data index. + if (trailer.getDataIndexCount() > 0) + BlockType.INDEX_V1.readAndCheck(dis); + dataBlockIndexReader.readRootIndex(dis, trailer.getDataIndexCount()); + + // Read in the metadata index. + if (trailer.getMetaIndexCount() > 0) + BlockType.INDEX_V1.readAndCheck(dis); + metaBlockIndexReader.readRootIndex(dis, trailer.getMetaIndexCount()); + + fileInfoLoaded = true; + return fileInfo; + } + + /** + * Creates comparator from the given class name. + * + * @param clazzName the comparator class name read from the trailer + * @return an instance of the comparator to use + * @throws IOException in case comparator class name is invalid + */ + @SuppressWarnings("unchecked") + private RawComparator getComparator(final String clazzName) + throws IOException { + if (clazzName == null || clazzName.length() == 0) { + return null; + } + try { + return (RawComparator)Class.forName(clazzName).newInstance(); + } catch (InstantiationException e) { + throw new IOException(e); + } catch (IllegalAccessException e) { + throw new IOException(e); + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + } + + /** + * Create a Scanner on this file. No seeks or reads are done on creation. Call + * {@link HFileScanner#seekTo(byte[])} to position an start the read. There is + * nothing to clean up in a Scanner. Letting go of your references to the + * scanner is sufficient. + * + * @param cacheBlocks True if we should cache blocks read in by this scanner. + * @param pread Use positional read rather than seek+read if true (pread is + * better for random reads, seek+read is better scanning). + * @param isCompaction is scanner being used for a compaction? + * @return Scanner on this file. + */ + @Override + public HFileScanner getScanner(boolean cacheBlocks, final boolean pread, + final boolean isCompaction) { + return new ScannerV1(this, cacheBlocks, pread, isCompaction); + } + + /** + * @param key Key to search. + * @return Block number of the block containing the key or -1 if not in this + * file. + */ + protected int blockContainingKey(final byte[] key, int offset, int length) { + Preconditions.checkState(!dataBlockIndexReader.isEmpty(), + "Block index not loaded"); + return dataBlockIndexReader.rootBlockContainingKey(key, offset, length); + } + + /** + * @param metaBlockName + * @param cacheBlock Add block to cache, if found + * @return Block wrapped in a ByteBuffer + * @throws IOException + */ + @Override + public ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock) + throws IOException { + if (trailer.getMetaIndexCount() == 0) { + return null; // there are no meta blocks + } + if (metaBlockIndexReader == null) { + throw new IOException("Meta index not loaded"); + } + + byte[] nameBytes = Bytes.toBytes(metaBlockName); + int block = metaBlockIndexReader.rootBlockContainingKey(nameBytes, 0, + nameBytes.length); + if (block == -1) + return null; + long offset = metaBlockIndexReader.getRootBlockOffset(block); + long nextOffset; + if (block == metaBlockIndexReader.getRootBlockCount() - 1) { + nextOffset = trailer.getFileInfoOffset(); + } else { + nextOffset = metaBlockIndexReader.getRootBlockOffset(block + 1); + } + + long startTimeNs = System.nanoTime(); + + String cacheKey = HFile.getBlockCacheKey(name, offset); + + // Per meta key from any given file, synchronize reads for said block + synchronized (metaBlockIndexReader.getRootBlockKey(block)) { + metaLoads.incrementAndGet(); + // Check cache for block. If found return. + if (blockCache != null) { + HFileBlock cachedBlock = (HFileBlock) blockCache.getBlock(cacheKey, + true); + if (cachedBlock != null) { + cacheHits.incrementAndGet(); + return cachedBlock.getBufferWithoutHeader(); + } + // Cache Miss, please load. + } + + HFileBlock hfileBlock = fsBlockReader.readBlockData(offset, + nextOffset - offset, metaBlockIndexReader.getRootBlockDataSize(block), + true); + hfileBlock.expectType(BlockType.META); + + HFile.readTimeNano.addAndGet(System.nanoTime() - startTimeNs); + HFile.readOps.incrementAndGet(); + + // Cache the block + if (cacheBlock && blockCache != null) { + blockCache.cacheBlock(cacheKey, hfileBlock, inMemory); + } + + return hfileBlock.getBufferWithoutHeader(); + } + } + + /** + * Read in a file block. + * @param block Index of block to read. + * @param pread Use positional read instead of seek+read (positional is + * better doing random reads whereas seek+read is better scanning). + * @param isCompaction is this block being read as part of a compaction + * @return Block wrapped in a ByteBuffer. + * @throws IOException + */ + ByteBuffer readBlockBuffer(int block, boolean cacheBlock, + final boolean pread, final boolean isCompaction) throws IOException { + if (dataBlockIndexReader == null) { + throw new IOException("Block index not loaded"); + } + if (block < 0 || block >= dataBlockIndexReader.getRootBlockCount()) { + throw new IOException("Requested block is out of range: " + block + + ", max: " + dataBlockIndexReader.getRootBlockCount()); + } + + long offset = dataBlockIndexReader.getRootBlockOffset(block); + String cacheKey = HFile.getBlockCacheKey(name, offset); + + // For any given block from any given file, synchronize reads for said + // block. + // Without a cache, this synchronizing is needless overhead, but really + // the other choice is to duplicate work (which the cache would prevent you + // from doing). + synchronized (dataBlockIndexReader.getRootBlockKey(block)) { + blockLoads.incrementAndGet(); + + // Check cache for block. If found return. + if (blockCache != null) { + HFileBlock cachedBlock = (HFileBlock) blockCache.getBlock(cacheKey, + true); + if (cachedBlock != null) { + cacheHits.incrementAndGet(); + return cachedBlock.getBufferWithoutHeader(); + } + // Carry on, please load. + } + + // Load block from filesystem. + long startTimeNs = System.nanoTime(); + long nextOffset; + + if (block == dataBlockIndexReader.getRootBlockCount() - 1) { + // last block! The end of data block is first meta block if there is + // one or if there isn't, the fileinfo offset. + nextOffset = (metaBlockIndexReader.getRootBlockCount() == 0) ? + this.trailer.getFileInfoOffset() : + metaBlockIndexReader.getRootBlockOffset(0); + } else { + nextOffset = dataBlockIndexReader.getRootBlockOffset(block + 1); + } + + HFileBlock hfileBlock = fsBlockReader.readBlockData(offset, nextOffset + - offset, dataBlockIndexReader.getRootBlockDataSize(block), pread); + hfileBlock.expectType(BlockType.DATA); + ByteBuffer buf = hfileBlock.getBufferWithoutHeader(); + + HFile.readTimeNano.addAndGet(System.nanoTime() - startTimeNs); + HFile.readOps.incrementAndGet(); + + // Cache the block + if (cacheBlock && blockCache != null) { + blockCache.cacheBlock(cacheKey, hfileBlock, inMemory); + } + + return buf; + } + } + + /** + * @return Last key in the file. May be null if file has no entries. + * Note that this is not the last rowkey, but rather the byte form of + * the last KeyValue. + */ + public byte[] getLastKey() { + if (!fileInfoLoaded) { + throw new RuntimeException("Load file info first"); + } + return dataBlockIndexReader.isEmpty() ? null : lastKey; + } + + /** + * @return Midkey for this file. We work with block boundaries only so + * returned midkey is an approximation only. + * + * @throws IOException + */ + @Override + public byte[] midkey() throws IOException { + Preconditions.checkState(isFileInfoLoaded(), "File info is not loaded"); + Preconditions.checkState(!dataBlockIndexReader.isEmpty(), + "Data block index is not loaded or is empty"); + return dataBlockIndexReader.midkey(); + } + + @Override + public void close() throws IOException { + if (evictOnClose && this.blockCache != null) { + int numEvicted = 0; + for (int i = 0; i < dataBlockIndexReader.getRootBlockCount(); i++) { + if (blockCache.evictBlock(HFile.getBlockCacheKey(name, + dataBlockIndexReader.getRootBlockOffset(i)))) + numEvicted++; + } + LOG.debug("On close of file " + name + " evicted " + numEvicted + + " block(s) of " + dataBlockIndexReader.getRootBlockCount() + + " total blocks"); + } + if (this.closeIStream && this.istream != null) { + this.istream.close(); + this.istream = null; + } + } + + /** + * Implementation of {@link HFileScanner} interface. + */ + protected static class ScannerV1 extends AbstractHFileReader.Scanner { + private final HFileReaderV1 readerV1; + private int currBlock; + + public ScannerV1(HFileReaderV1 reader, boolean cacheBlocks, + final boolean pread, final boolean isCompaction) { + super(reader, cacheBlocks, pread, isCompaction); + readerV1 = reader; + } + + @Override + public KeyValue getKeyValue() { + if (blockBuffer == null) { + return null; + } + return new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset() + + blockBuffer.position() - 8); + } + + @Override + public ByteBuffer getKey() { + Preconditions.checkState(blockBuffer != null && currKeyLen > 0, + "you need to seekTo() before calling getKey()"); + + ByteBuffer keyBuff = blockBuffer.slice(); + keyBuff.limit(currKeyLen); + keyBuff.rewind(); + // Do keyBuff.asReadOnly()? + return keyBuff; + } + + @Override + public ByteBuffer getValue() { + if (blockBuffer == null || currKeyLen == 0) { + throw new RuntimeException( + "you need to seekTo() before calling getValue()"); + } + + // TODO: Could this be done with one ByteBuffer rather than create two? + ByteBuffer valueBuff = blockBuffer.slice(); + valueBuff.position(currKeyLen); + valueBuff = valueBuff.slice(); + valueBuff.limit(currValueLen); + valueBuff.rewind(); + return valueBuff; + } + + @Override + public boolean next() throws IOException { + if (blockBuffer == null) { + throw new IOException("Next called on non-seeked scanner"); + } + + try { + blockBuffer.position(blockBuffer.position() + currKeyLen + + currValueLen); + } catch (IllegalArgumentException e) { + LOG.error("Current pos = " + blockBuffer.position() + + "; currKeyLen = " + currKeyLen + + "; currValLen = " + currValueLen + + "; block limit = " + blockBuffer.limit() + + "; HFile name = " + reader.getName() + + "; currBlock id = " + currBlock, e); + throw e; + } + if (blockBuffer.remaining() <= 0) { + // LOG.debug("Fetch next block"); + currBlock++; + if (currBlock >= reader.getDataBlockIndexReader().getRootBlockCount()) { + // damn we are at the end + currBlock = 0; + blockBuffer = null; + return false; + } + blockBuffer = readerV1.readBlockBuffer(currBlock, cacheBlocks, pread, + isCompaction); + currKeyLen = blockBuffer.getInt(); + currValueLen = blockBuffer.getInt(); + blockFetches++; + return true; + } + + currKeyLen = blockBuffer.getInt(); + currValueLen = blockBuffer.getInt(); + return true; + } + + @Override + public int seekTo(byte[] key) throws IOException { + return seekTo(key, 0, key.length); + } + + @Override + public int seekTo(byte[] key, int offset, int length) throws IOException { + int b = readerV1.blockContainingKey(key, offset, length); + if (b < 0) return -1; // falls before the beginning of the file! :-( + // Avoid re-reading the same block (that'd be dumb). + loadBlock(b, true); + return blockSeek(key, offset, length, false); + } + + @Override + public int reseekTo(byte[] key) throws IOException { + return reseekTo(key, 0, key.length); + } + + @Override + public int reseekTo(byte[] key, int offset, int length) + throws IOException { + if (blockBuffer != null && currKeyLen != 0) { + ByteBuffer bb = getKey(); + int compared = reader.getComparator().compare(key, offset, + length, bb.array(), bb.arrayOffset(), bb.limit()); + if (compared <= 0) { + // If the required key is less than or equal to current key, then + // don't do anything. + return compared; + } + } + + int b = readerV1.blockContainingKey(key, offset, length); + if (b < 0) { + return -1; + } + loadBlock(b, false); + return blockSeek(key, offset, length, false); + } + + /** + * Within a loaded block, seek looking for the first key + * that is smaller than (or equal to?) the key we are interested in. + * + * A note on the seekBefore - if you have seekBefore = true, AND the + * first key in the block = key, then you'll get thrown exceptions. + * @param key to find + * @param seekBefore find the key before the exact match. + * @return + */ + private int blockSeek(byte[] key, int offset, int length, + boolean seekBefore) { + int klen, vlen; + int lastLen = 0; + do { + klen = blockBuffer.getInt(); + vlen = blockBuffer.getInt(); + int comp = reader.getComparator().compare(key, offset, length, + blockBuffer.array(), + blockBuffer.arrayOffset() + blockBuffer.position(), klen); + if (comp == 0) { + if (seekBefore) { + blockBuffer.position(blockBuffer.position() - lastLen - 16); + currKeyLen = blockBuffer.getInt(); + currValueLen = blockBuffer.getInt(); + return 1; // non exact match. + } + currKeyLen = klen; + currValueLen = vlen; + return 0; // indicate exact match + } + if (comp < 0) { + // go back one key: + blockBuffer.position(blockBuffer.position() - lastLen - 16); + currKeyLen = blockBuffer.getInt(); + currValueLen = blockBuffer.getInt(); + return 1; + } + blockBuffer.position(blockBuffer.position() + klen + vlen); + lastLen = klen + vlen; + } while (blockBuffer.remaining() > 0); + + // ok we are at the end, so go back a littleeeeee.... + // The 8 in the below is intentionally different to the 16s in the above + // Do the math you you'll figure it. + blockBuffer.position(blockBuffer.position() - lastLen - 8); + currKeyLen = blockBuffer.getInt(); + currValueLen = blockBuffer.getInt(); + return 1; // didn't exactly find it. + } + + @Override + public boolean seekBefore(byte[] key) throws IOException { + return seekBefore(key, 0, key.length); + } + + @Override + public boolean seekBefore(byte[] key, int offset, int length) + throws IOException { + int b = readerV1.blockContainingKey(key, offset, length); + if (b < 0) + return false; // key is before the start of the file. + + // Question: does this block begin with 'key'? + byte[] firstkKey = reader.getDataBlockIndexReader().getRootBlockKey(b); + if (reader.getComparator().compare(firstkKey, 0, firstkKey.length, + key, offset, length) == 0) { + // Ok the key we're interested in is the first of the block, so go back + // by one. + if (b == 0) { + // we have a 'problem', the key we want is the first of the file. + return false; + } + b--; + // TODO shortcut: seek forward in this block to the last key of the + // block. + } + loadBlock(b, true); + blockSeek(key, offset, length, true); + return true; + } + + @Override + public String getKeyString() { + return Bytes.toStringBinary(blockBuffer.array(), + blockBuffer.arrayOffset() + blockBuffer.position(), currKeyLen); + } + + @Override + public String getValueString() { + return Bytes.toString(blockBuffer.array(), blockBuffer.arrayOffset() + + blockBuffer.position() + currKeyLen, currValueLen); + } + + @Override + public Reader getReader() { + return reader; + } + + @Override + public boolean seekTo() throws IOException { + if (reader.getDataBlockIndexReader().isEmpty()) { + return false; + } + if (blockBuffer != null && currBlock == 0) { + blockBuffer.rewind(); + currKeyLen = blockBuffer.getInt(); + currValueLen = blockBuffer.getInt(); + return true; + } + currBlock = 0; + blockBuffer = readerV1.readBlockBuffer(currBlock, cacheBlocks, pread, + isCompaction); + currKeyLen = blockBuffer.getInt(); + currValueLen = blockBuffer.getInt(); + blockFetches++; + return true; + } + + private void loadBlock(int bloc, boolean rewind) throws IOException { + if (blockBuffer == null) { + blockBuffer = readerV1.readBlockBuffer(bloc, cacheBlocks, pread, + isCompaction); + currBlock = bloc; + blockFetches++; + } else { + if (bloc != currBlock) { + blockBuffer = readerV1.readBlockBuffer(bloc, cacheBlocks, pread, + isCompaction); + currBlock = bloc; + blockFetches++; + } else { + // we are already in the same block, just rewind to seek again. + if (rewind) { + blockBuffer.rewind(); + } + else { + // Go back by (size of rowlength + size of valuelength) = 8 bytes + blockBuffer.position(blockBuffer.position()-8); + } + } + } + } + + } + + @Override + public HFileBlock readBlock(long offset, int onDiskBlockSize, + boolean cacheBlock, boolean pread, boolean isCompaction) { + throw new UnsupportedOperationException(); + } + + @Override + public DataInput getBloomFilterMetadata() throws IOException { + ByteBuffer buf = getMetaBlock(HFileWriterV1.BLOOM_FILTER_META_KEY, false); + if (buf == null) + return null; + ByteArrayInputStream bais = new ByteArrayInputStream(buf.array(), + buf.arrayOffset(), buf.limit()); + return new DataInputStream(bais); + } + + @Override + public boolean isFileInfoLoaded() { + return fileInfoLoaded; + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java new file mode 100644 index 0000000..1cf7767 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java @@ -0,0 +1,732 @@ +/* + * Copyright 2011 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.DataInput; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.IdLock; + +/** + * {@link HFile} reader for version 2. + */ +public class HFileReaderV2 extends AbstractHFileReader implements + HFileBlock.BasicReader { + + private static final Log LOG = LogFactory.getLog(HFileReaderV2.class); + + /** + * The size of a (key length, value length) tuple that prefixes each entry in + * a data block. + */ + private static final int KEY_VALUE_LEN_SIZE = 2 * Bytes.SIZEOF_INT; + + /** + * A "sparse lock" implementation allowing to lock on a particular block + * identified by offset. The purpose of this is to avoid two clients loading + * the same block, and have all but one client wait to get the block from the + * cache. + */ + private IdLock offsetLock = new IdLock(); + + /** + * Blocks read from the load-on-open section, excluding data root index, meta + * index, and file info. + */ + private List loadOnOpenBlocks = new ArrayList(); + + /** + * Opens a HFile. You must load the index before you can use it by calling + * {@link #loadFileInfo()}. + * + * @param fsdis input stream. Caller is responsible for closing the passed + * stream. + * @param size Length of the stream. + * @param blockCache block cache. Pass null if none. + * @param inMemory whether blocks should be marked as in-memory in cache + * @param evictOnClose whether blocks in cache should be evicted on close + * @throws IOException + */ + public HFileReaderV2(Path path, FixedFileTrailer trailer, + final FSDataInputStream fsdis, final long size, + final boolean closeIStream, final BlockCache blockCache, + final boolean inMemory, final boolean evictOnClose) throws IOException { + super(path, trailer, fsdis, size, closeIStream, blockCache, inMemory, + evictOnClose); + + trailer.expectVersion(2); + fsBlockReader = new HFileBlock.FSReaderV2(fsdis, compressAlgo, + fileSize); + + // Comparator class name is stored in the trailer in version 2. + comparator = trailer.createComparator(); + dataBlockIndexReader = new HFileBlockIndex.BlockIndexReader(comparator, + trailer.getNumDataIndexLevels(), this); + metaBlockIndexReader = new HFileBlockIndex.BlockIndexReader( + Bytes.BYTES_RAWCOMPARATOR, 1); + + // Parse load-on-open data. + + HFileBlock.BlockIterator blockIter = fsBlockReader.blockRange( + trailer.getLoadOnOpenDataOffset(), + fileSize - trailer.getTrailerSize()); + + // Data index. We also read statistics about the block index written after + // the root level. + dataBlockIndexReader.readMultiLevelIndexRoot( + blockIter.nextBlockAsStream(BlockType.ROOT_INDEX), + trailer.getDataIndexCount()); + + // Meta index. + metaBlockIndexReader.readRootIndex( + blockIter.nextBlockAsStream(BlockType.ROOT_INDEX), + trailer.getMetaIndexCount()); + + // File info + fileInfo = new FileInfo(); + fileInfo.readFields(blockIter.nextBlockAsStream(BlockType.FILE_INFO)); + lastKey = fileInfo.get(FileInfo.LASTKEY); + avgKeyLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_KEY_LEN)); + avgValueLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_VALUE_LEN)); + + // Store all other load-on-open blocks for further consumption. + HFileBlock b; + while ((b = blockIter.nextBlock()) != null) { + loadOnOpenBlocks.add(b); + } + } + + /** + * Create a Scanner on this file. No seeks or reads are done on creation. Call + * {@link HFileScanner#seekTo(byte[])} to position an start the read. There is + * nothing to clean up in a Scanner. Letting go of your references to the + * scanner is sufficient. + * + * @param cacheBlocks True if we should cache blocks read in by this scanner. + * @param pread Use positional read rather than seek+read if true (pread is + * better for random reads, seek+read is better scanning). + * @param isCompaction is scanner being used for a compaction? + * @return Scanner on this file. + */ + @Override + public HFileScanner getScanner(boolean cacheBlocks, final boolean pread, + final boolean isCompaction) { + return new ScannerV2(this, cacheBlocks, pread, isCompaction); + } + + /** + * @param metaBlockName + * @param cacheBlock Add block to cache, if found + * @return block wrapped in a ByteBuffer, with header skipped + * @throws IOException + */ + @Override + public ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock) + throws IOException { + if (trailer.getMetaIndexCount() == 0) { + return null; // there are no meta blocks + } + if (metaBlockIndexReader == null) { + throw new IOException("Meta index not loaded"); + } + + byte[] mbname = Bytes.toBytes(metaBlockName); + int block = metaBlockIndexReader.rootBlockContainingKey(mbname, 0, + mbname.length); + if (block == -1) + return null; + long blockSize = metaBlockIndexReader.getRootBlockDataSize(block); + long startTimeNs = System.nanoTime(); + + // Per meta key from any given file, synchronize reads for said block. This + // is OK to do for meta blocks because the meta block index is always + // single-level. + synchronized (metaBlockIndexReader.getRootBlockKey(block)) { + metaLoads.incrementAndGet(); + + // Check cache for block. If found return. + long metaBlockOffset = metaBlockIndexReader.getRootBlockOffset(block); + String cacheKey = HFile.getBlockCacheKey(name, metaBlockOffset); + + if (blockCache != null) { + HFileBlock cachedBlock = (HFileBlock) blockCache.getBlock(cacheKey, + true); + if (cachedBlock != null) { + // Return a distinct 'shallow copy' of the block, + // so pos does not get messed by the scanner + cacheHits.incrementAndGet(); + return cachedBlock.getBufferWithoutHeader(); + } + // Cache Miss, please load. + } + + HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset, + blockSize, -1, true); + + HFile.readTimeNano.addAndGet(System.nanoTime() - startTimeNs); + HFile.readOps.incrementAndGet(); + + // Cache the block + if (cacheBlock && blockCache != null) { + blockCache.cacheBlock(cacheKey, metaBlock, inMemory); + } + + return metaBlock.getBufferWithoutHeader(); + } + } + + /** + * Implements the "basic block reader" API, used mainly by + * {@link HFileBlockIndex.BlockIndexReader} in + * {@link HFileBlockIndex.BlockIndexReader#seekToDataBlock(byte[], int, int, + * HFileBlock)} in a random-read access pattern. + */ + @Override + public HFileBlock readBlockData(long offset, long onDiskSize, + int uncompressedSize, boolean pread) throws IOException { + if (onDiskSize >= Integer.MAX_VALUE) { + throw new IOException("Invalid on-disk size: " + onDiskSize); + } + + // Assuming we are not doing a compaction. + return readBlock(offset, (int) onDiskSize, true, pread, false); + } + + /** + * Read in a file block. + * + * @param dataBlockOffset offset to read. + * @param onDiskSize size of the block + * @param pread Use positional read instead of seek+read (positional is better + * doing random reads whereas seek+read is better scanning). + * @param isCompaction is this block being read as part of a compaction + * @return Block wrapped in a ByteBuffer. + * @throws IOException + */ + @Override + public HFileBlock readBlock(long dataBlockOffset, int onDiskBlockSize, + boolean cacheBlock, boolean pread, final boolean isCompaction) + throws IOException { + if (dataBlockIndexReader == null) { + throw new IOException("Block index not loaded"); + } + if (dataBlockOffset < 0 + || dataBlockOffset >= trailer.getLoadOnOpenDataOffset()) { + throw new IOException("Requested block is out of range: " + + dataBlockOffset + ", lastDataBlockOffset: " + + trailer.getLastDataBlockOffset()); + } + // For any given block from any given file, synchronize reads for said + // block. + // Without a cache, this synchronizing is needless overhead, but really + // the other choice is to duplicate work (which the cache would prevent you + // from doing). + + String cacheKey = HFile.getBlockCacheKey(name, dataBlockOffset); + IdLock.Entry lockEntry = offsetLock.getLockEntry(dataBlockOffset); + try { + blockLoads.incrementAndGet(); + + // Check cache for block. If found return. + if (blockCache != null) { + HFileBlock cachedBlock = (HFileBlock) blockCache.getBlock(cacheKey, + true); + if (cachedBlock != null) { + cacheHits.incrementAndGet(); + + return cachedBlock; + } + // Carry on, please load. + } + + // Load block from filesystem. + long startTimeNs = System.nanoTime(); + HFileBlock dataBlock = fsBlockReader.readBlockData(dataBlockOffset, + onDiskBlockSize, -1, pread); + + HFile.readTimeNano.addAndGet(System.nanoTime() - startTimeNs); + HFile.readOps.incrementAndGet(); + + // Cache the block + if (cacheBlock && blockCache != null) { + blockCache.cacheBlock(cacheKey, dataBlock, inMemory); + } + + return dataBlock; + } finally { + offsetLock.releaseLockEntry(lockEntry); + } + } + + /** + * @return Last key in the file. May be null if file has no entries. Note that + * this is not the last row key, but rather the byte form of the last + * KeyValue. + */ + @Override + public byte[] getLastKey() { + return dataBlockIndexReader.isEmpty() ? null : lastKey; + } + + /** + * @return Midkey for this file. We work with block boundaries only so + * returned midkey is an approximation only. + * @throws IOException + */ + @Override + public byte[] midkey() throws IOException { + return dataBlockIndexReader.midkey(); + } + + @Override + public void close() throws IOException { + if (evictOnClose && blockCache != null) { + int numEvicted = blockCache.evictBlocksByPrefix(name + + HFile.CACHE_KEY_SEPARATOR); + LOG.debug("On close of file " + name + " evicted " + numEvicted + + " block(s)"); + } + if (closeIStream && istream != null) { + istream.close(); + istream = null; + } + } + + /** + * Implementation of {@link HFileScanner} interface. + */ + protected static class ScannerV2 extends AbstractHFileReader.Scanner { + private HFileBlock block; + + public ScannerV2(HFileReaderV2 r, boolean cacheBlocks, + final boolean pread, final boolean isCompaction) { + super(r, cacheBlocks, pread, isCompaction); + } + + @Override + public KeyValue getKeyValue() { + if (!isSeeked()) + return null; + + return new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset() + + blockBuffer.position()); + } + + @Override + public ByteBuffer getKey() { + assertSeeked(); + return ByteBuffer.wrap( + blockBuffer.array(), + blockBuffer.arrayOffset() + blockBuffer.position() + + KEY_VALUE_LEN_SIZE, currKeyLen).slice(); + } + + @Override + public ByteBuffer getValue() { + assertSeeked(); + return ByteBuffer.wrap( + blockBuffer.array(), + blockBuffer.arrayOffset() + blockBuffer.position() + + KEY_VALUE_LEN_SIZE + currKeyLen, currValueLen).slice(); + } + + private void setNonSeekedState() { + block = null; + blockBuffer = null; + currKeyLen = 0; + currValueLen = 0; + } + + /** + * Go to the next key/value in the block section. Loads the next block if + * necessary. If successful, {@link #getKey()} and {@link #getValue()} can + * be called. + * + * @return true if successfully navigated to the next key/value + */ + @Override + public boolean next() throws IOException { + assertSeeked(); + + try { + blockBuffer.position(blockBuffer.position() + KEY_VALUE_LEN_SIZE + + currKeyLen + currValueLen); + } catch (IllegalArgumentException e) { + LOG.error("Current pos = " + blockBuffer.position() + + "; currKeyLen = " + currKeyLen + "; currValLen = " + + currValueLen + "; block limit = " + blockBuffer.limit() + + "; HFile name = " + reader.getName() + + "; currBlock currBlockOffset = " + block.getOffset()); + throw e; + } + + if (blockBuffer.remaining() <= 0) { + long lastDataBlockOffset = + reader.getTrailer().getLastDataBlockOffset(); + + if (block.getOffset() >= lastDataBlockOffset) { + setNonSeekedState(); + return false; + } + + // read the next block + HFileBlock nextBlock = readNextDataBlock(); + if (nextBlock == null) { + setNonSeekedState(); + return false; + } + + updateCurrBlock(nextBlock); + return true; + } + + // We are still in the same block. + readKeyValueLen(); + return true; + } + + /** + * Scans blocks in the "scanned" section of the {@link HFile} until the next + * data block is found. + * + * @return the next block, or null if there are no more data blocks + * @throws IOException + */ + private HFileBlock readNextDataBlock() throws IOException { + long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset(); + if (block == null) + return null; + + HFileBlock curBlock = block; + + do { + if (curBlock.getOffset() >= lastDataBlockOffset) + return null; + + if (curBlock.getOffset() < 0) { + throw new IOException("Invalid block file offset: " + block); + } + curBlock = reader.readBlock(curBlock.getOffset() + + curBlock.getOnDiskSizeWithHeader(), + curBlock.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread, + isCompaction); + } while (!curBlock.getBlockType().equals(BlockType.DATA)); + + return curBlock; + } + + /** + * 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 + */ + @Override + public boolean seekTo() throws IOException { + if (reader == null) { + return false; + } + + if (reader.getTrailer().getEntryCount() == 0) { + // No data blocks. + return false; + } + + long firstDataBlockOffset = + reader.getTrailer().getFirstDataBlockOffset(); + if (block != null && block.getOffset() == firstDataBlockOffset) { + blockBuffer.rewind(); + readKeyValueLen(); + return true; + } + + block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread, + isCompaction); + if (block.getOffset() < 0) { + throw new IOException("Invalid block offset: " + block.getOffset()); + } + updateCurrBlock(block); + return true; + } + + @Override + public int seekTo(byte[] key) throws IOException { + return seekTo(key, 0, key.length); + } + + /** + * An internal API function. Seek to the given key, optionally rewinding to + * the first key of the block before doing the seek. + * + * @param key key byte array + * @param offset key offset in the key byte array + * @param length key length + * @param rewind whether to rewind to the first key of the block before + * doing the seek. If this is false, we are assuming we never go + * back, otherwise the result is undefined. + * @return -1 if the key is earlier than the first key of the file, + * 0 if we are at the given key, and 1 if we are past the given key + * @throws IOException + */ + private int seekTo(byte[] key, int offset, int length, boolean rewind) + throws IOException { + HFileBlock seekToBlock = + ((HFileReaderV2) reader).getDataBlockIndexReader().seekToDataBlock( + key, offset, length, block); + if (seekToBlock == null) { + // This happens if the key e.g. falls before the beginning of the file. + return -1; + } + return loadBlockAndSeekToKey(seekToBlock, rewind, key, offset, length, + false); + } + + @Override + public int seekTo(byte[] key, int offset, int length) throws IOException { + // Always rewind to the first key of the block, because the given key + // might be before or after the current key. + return seekTo(key, offset, length, true); + } + + @Override + public int reseekTo(byte[] key) throws IOException { + return reseekTo(key, 0, key.length); + } + + @Override + public int reseekTo(byte[] key, int offset, int length) throws IOException { + if (isSeeked()) { + ByteBuffer bb = getKey(); + int compared = reader.getComparator().compare(key, offset, + length, bb.array(), bb.arrayOffset(), bb.limit()); + if (compared < 1) { + // If the required key is less than or equal to current key, then + // don't do anything. + return compared; + } + } + + // Don't rewind on a reseek operation, because reseek implies that we are + // always going forward in the file. + return seekTo(key, offset, length, false); + } + + private int loadBlockAndSeekToKey(HFileBlock seekToBlock, boolean rewind, + byte[] key, int offset, int length, boolean seekBefore) + throws IOException { + if (block == null || block.getOffset() != seekToBlock.getOffset()) { + updateCurrBlock(seekToBlock); + } else if (rewind) { + blockBuffer.rewind(); + } + return blockSeek(key, offset, length, seekBefore); + } + + /** + * Updates the current block to be the given {@link HFileBlock}. Seeks to + * the the first key/value pair. + * + * @param newBlock the block to make current + */ + private void updateCurrBlock(HFileBlock newBlock) { + block = newBlock; + blockBuffer = block.getBufferWithoutHeader(); + readKeyValueLen(); + blockFetches++; + } + + private final void readKeyValueLen() { + blockBuffer.mark(); + currKeyLen = blockBuffer.getInt(); + currValueLen = blockBuffer.getInt(); + blockBuffer.reset(); + + if (currKeyLen < 0 || currValueLen < 0 + || currKeyLen > blockBuffer.limit() + || currValueLen > blockBuffer.limit()) { + throw new IllegalStateException("Invalid currKeyLen " + currKeyLen + + " or currValueLen " + currValueLen + ". Block offset: " + + block.getOffset() + ", block length: " + blockBuffer.limit() + + ", position: " + blockBuffer.position() + " (without header)."); + } + } + + /** + * Within a loaded block, seek looking for the first key that is smaller + * than (or equal to?) the key we are interested in. + * + * A note on the seekBefore: if you have seekBefore = true, AND the first + * key in the block = key, then you'll get thrown exceptions. The caller has + * to check for that case and load the previous block as appropriate. + * + * @param key the key to find + * @param seekBefore find the key before the given key in case of exact + * match. + * @return 0 in case of an exact key match, 1 in case of an inexact match + */ + private int blockSeek(byte[] key, int offset, int length, + boolean seekBefore) { + int klen, vlen; + int lastKeyValueSize = -1; + do { + blockBuffer.mark(); + klen = blockBuffer.getInt(); + vlen = blockBuffer.getInt(); + blockBuffer.reset(); + + int keyOffset = blockBuffer.arrayOffset() + blockBuffer.position() + + KEY_VALUE_LEN_SIZE; + int comp = reader.getComparator().compare(key, offset, length, + blockBuffer.array(), keyOffset, klen); + + if (comp == 0) { + if (seekBefore) { + if (lastKeyValueSize < 0) { + throw new IllegalStateException("blockSeek with seekBefore " + + "at the first key of the block: key=" + + Bytes.toStringBinary(key) + ", blockOffset=" + + block.getOffset() + ", onDiskSize=" + + block.getOnDiskSizeWithHeader()); + } + blockBuffer.position(blockBuffer.position() - lastKeyValueSize); + readKeyValueLen(); + return 1; // non exact match. + } + currKeyLen = klen; + currValueLen = vlen; + return 0; // indicate exact match + } + + if (comp < 0) { + if (lastKeyValueSize > 0) + blockBuffer.position(blockBuffer.position() - lastKeyValueSize); + readKeyValueLen(); + return 1; + } + + // The size of this key/value tuple, including key/value length fields. + lastKeyValueSize = klen + vlen + KEY_VALUE_LEN_SIZE; + blockBuffer.position(blockBuffer.position() + lastKeyValueSize); + } while (blockBuffer.remaining() > 0); + + // Seek to the last key we successfully read. This will happen if this is + // the last key/value pair in the file, in which case the following call + // to next() has to return false. + blockBuffer.position(blockBuffer.position() - lastKeyValueSize); + readKeyValueLen(); + return 1; // didn't exactly find it. + } + + @Override + public boolean seekBefore(byte[] key) throws IOException { + return seekBefore(key, 0, key.length); + } + + private ByteBuffer getFirstKeyInBlock(HFileBlock curBlock) { + ByteBuffer buffer = curBlock.getBufferWithoutHeader(); + // It is safe to manipulate this buffer because we own the buffer object. + buffer.rewind(); + int klen = buffer.getInt(); + buffer.getInt(); + ByteBuffer keyBuff = buffer.slice(); + keyBuff.limit(klen); + keyBuff.rewind(); + return keyBuff; + } + + @Override + public boolean seekBefore(byte[] key, int offset, int length) + throws IOException { + HFileReaderV2 reader2 = (HFileReaderV2) reader; + HFileBlock seekToBlock = + reader2.getDataBlockIndexReader().seekToDataBlock( + key, offset, length, block); + if (seekToBlock == null) { + return false; + } + ByteBuffer firstKey = getFirstKeyInBlock(seekToBlock); + if (reader.getComparator().compare(firstKey.array(), + firstKey.arrayOffset(), firstKey.limit(), key, offset, length) == 0) + { + long previousBlockOffset = seekToBlock.getPrevBlockOffset(); + // The key we are interested in + if (previousBlockOffset == -1) { + // we have a 'problem', the key we want is the first of the file. + return false; + } + + // It is important that we compute and pass onDiskSize to the block + // reader so that it does not have to read the header separately to + // figure out the size. + seekToBlock = reader2.fsBlockReader.readBlockData(previousBlockOffset, + seekToBlock.getOffset() - previousBlockOffset, -1, pread); + + // TODO shortcut: seek forward in this block to the last key of the + // block. + } + loadBlockAndSeekToKey(seekToBlock, true, key, offset, length, true); + return true; + } + + @Override + public String getKeyString() { + return Bytes.toStringBinary(blockBuffer.array(), + blockBuffer.arrayOffset() + blockBuffer.position() + + KEY_VALUE_LEN_SIZE, currKeyLen); + } + + @Override + public String getValueString() { + return Bytes.toString(blockBuffer.array(), blockBuffer.arrayOffset() + + blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen, + currValueLen); + } + + } + + /** + * Returns a buffer with the Bloom filter metadata. The caller takes + * ownership of the buffer. + */ + @Override + public DataInput getBloomFilterMetadata() throws IOException { + for (HFileBlock b : loadOnOpenBlocks) + if (b.getBlockType() == BlockType.BLOOM_META) + return b.getByteStream(); + return null; + } + + @Override + public boolean isFileInfoLoaded() { + return true; // We load file info in constructor in version 2. + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java new file mode 100644 index 0000000..443000f --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java @@ -0,0 +1,483 @@ +/* + * Copyright 2011 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.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.KeyComparator; +import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; +import org.apache.hadoop.hbase.io.hfile.HFile.Writer; +import org.apache.hadoop.hbase.util.BloomFilterWriter; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.compress.Compressor; + +/** + * Writes version 1 HFiles. Mainly used for testing backwards-compatibilty. + */ +public class HFileWriterV1 extends AbstractHFileWriter { + + /** Meta data block name for bloom filter parameters. */ + static final String BLOOM_FILTER_META_KEY = "BLOOM_FILTER_META"; + + /** Meta data block name for bloom filter bits. */ + public static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA"; + + private static final Log LOG = LogFactory.getLog(HFileWriterV1.class); + + // A stream made per block written. + private DataOutputStream out; + + // Offset where the current block began. + private long blockBegin; + + // First keys of every block. + private ArrayList blockKeys = new ArrayList(); + + // Block offset in backing stream. + private ArrayList blockOffsets = new ArrayList(); + + // Raw (decompressed) data size. + private ArrayList blockDataSizes = new ArrayList(); + + private Compressor compressor; + + // Additional byte array output stream used to fill block cache + private ByteArrayOutputStream baos; + private DataOutputStream baosDos; + private int blockNumber = 0; + + static class WriterFactoryV1 extends HFile.WriterFactory { + + WriterFactoryV1(Configuration conf) { super(conf); } + + @Override + public Writer createWriter(FileSystem fs, Path path) throws IOException { + return new HFileWriterV1(conf, fs, path); + } + + @Override + public Writer createWriter(FileSystem fs, Path path, int blockSize, + Compression.Algorithm compressAlgo, final KeyComparator comparator) + throws IOException { + return new HFileWriterV1(conf, fs, path, blockSize, + compressAlgo, comparator); + } + + @Override + public Writer createWriter(FileSystem fs, Path path, int blockSize, + String compressAlgoName, + final KeyComparator comparator) throws IOException { + return new HFileWriterV1(conf, fs, path, blockSize, + compressAlgoName, comparator); + } + + @Override + public Writer createWriter(final FSDataOutputStream ostream, + final int blockSize, final String compress, + final KeyComparator comparator) throws IOException { + return new HFileWriterV1(conf, ostream, blockSize, compress, comparator); + } + + @Override + public Writer createWriter(final FSDataOutputStream ostream, + final int blockSize, final Compression.Algorithm compress, + final KeyComparator c) throws IOException { + return new HFileWriterV1(conf, ostream, blockSize, compress, c); + } + } + + /** Constructor that uses all defaults for compression and block size. */ + public HFileWriterV1(Configuration conf, FileSystem fs, Path path) + throws IOException { + this(conf, fs, path, HFile.DEFAULT_BLOCKSIZE, + HFile.DEFAULT_COMPRESSION_ALGORITHM, + null); + } + + /** + * Constructor that takes a path, creates and closes the output stream. Takes + * compression algorithm name as string. + */ + public HFileWriterV1(Configuration conf, FileSystem fs, Path path, + int blockSize, String compressAlgoName, + final KeyComparator comparator) throws IOException { + this(conf, fs, path, blockSize, + compressionByName(compressAlgoName), comparator); + } + + /** Constructor that takes a path, creates and closes the output stream. */ + public HFileWriterV1(Configuration conf, FileSystem fs, Path path, + int blockSize, Compression.Algorithm compress, + final KeyComparator comparator) throws IOException { + super(conf, createOutputStream(conf, fs, path), path, + blockSize, compress, comparator); + } + + /** Constructor that takes a stream. */ + public HFileWriterV1(Configuration conf, + final FSDataOutputStream outputStream, final int blockSize, + final String compressAlgoName, final KeyComparator comparator) + throws IOException { + this(conf, outputStream, blockSize, + Compression.getCompressionAlgorithmByName(compressAlgoName), + comparator); + } + + /** Constructor that takes a stream. */ + public HFileWriterV1(Configuration conf, + final FSDataOutputStream outputStream, final int blockSize, + final Compression.Algorithm compress, final KeyComparator comparator) + throws IOException { + super(conf, outputStream, null, blockSize, compress, comparator); + } + + /** + * If at block boundary, opens new block. + * + * @throws IOException + */ + private void checkBlockBoundary() throws IOException { + if (this.out != null && this.out.size() < blockSize) + return; + finishBlock(); + newBlock(); + } + + /** + * Do the cleanup if a current block. + * + * @throws IOException + */ + private void finishBlock() throws IOException { + if (this.out == null) + return; + long startTimeNs = System.nanoTime(); + + int size = releaseCompressingStream(this.out); + this.out = null; + blockKeys.add(firstKeyInBlock); + blockOffsets.add(Long.valueOf(blockBegin)); + blockDataSizes.add(Integer.valueOf(size)); + this.totalUncompressedBytes += size; + + HFile.writeTimeNano.addAndGet(System.nanoTime() - startTimeNs); + HFile.writeOps.incrementAndGet(); + + if (cacheDataBlocksOnWrite) { + baosDos.flush(); + byte[] bytes = baos.toByteArray(); + blockCache.cacheBlock(HFile.getBlockCacheKey(name, blockBegin), + new HFileBlock(BlockType.DATA, + (int) (outputStream.getPos() - blockBegin), bytes.length, -1, + ByteBuffer.wrap(bytes, 0, bytes.length), true, blockBegin)); + baosDos.close(); + } + blockNumber++; + } + + /** + * Ready a new block for writing. + * + * @throws IOException + */ + private void newBlock() throws IOException { + // This is where the next block begins. + blockBegin = outputStream.getPos(); + this.out = getCompressingStream(); + BlockType.DATA.write(out); + firstKeyInBlock = null; + if (cacheDataBlocksOnWrite) { + this.baos = new ByteArrayOutputStream(); + this.baosDos = new DataOutputStream(baos); + baosDos.write(HFileBlock.DUMMY_HEADER); + } + } + + /** + * Sets up a compressor and creates a compression stream on top of + * this.outputStream. Get one per block written. + * + * @return A compressing stream; if 'none' compression, returned stream does + * not compress. + * + * @throws IOException + * + * @see {@link #releaseCompressingStream(DataOutputStream)} + */ + private DataOutputStream getCompressingStream() throws IOException { + this.compressor = compressAlgo.getCompressor(); + // Get new DOS compression stream. In tfile, the DOS, is not closed, + // just finished, and that seems to be fine over there. TODO: Check + // no memory retention of the DOS. Should I disable the 'flush' on the + // DOS as the BCFile over in tfile does? It wants to make it so flushes + // don't go through to the underlying compressed stream. Flush on the + // compressed downstream should be only when done. I was going to but + // looks like when we call flush in here, its legitimate flush that + // should go through to the compressor. + OutputStream os = this.compressAlgo.createCompressionStream( + this.outputStream, this.compressor, 0); + return new DataOutputStream(os); + } + + /** + * Let go of block compressor and compressing stream gotten in call {@link + * #getCompressingStream}. + * + * @param dos + * + * @return How much was written on this stream since it was taken out. + * + * @see #getCompressingStream() + * + * @throws IOException + */ + private int releaseCompressingStream(final DataOutputStream dos) + throws IOException { + dos.flush(); + this.compressAlgo.returnCompressor(this.compressor); + this.compressor = null; + return dos.size(); + } + + /** + * Add a meta block to the end of the file. Call before close(). Metadata + * blocks are expensive. Fill one with a bunch of serialized data rather than + * do a metadata block per metadata instance. If metadata is small, consider + * adding to file info using {@link #appendFileInfo(byte[], byte[])} + * + * @param metaBlockName + * name of the block + * @param content + * will call readFields to get data later (DO NOT REUSE) + */ + public void appendMetaBlock(String metaBlockName, Writable content) { + byte[] key = Bytes.toBytes(metaBlockName); + int i; + for (i = 0; i < metaNames.size(); ++i) { + // stop when the current key is greater than our own + byte[] cur = metaNames.get(i); + if (Bytes.BYTES_RAWCOMPARATOR.compare(cur, 0, cur.length, key, 0, + key.length) > 0) { + break; + } + } + metaNames.add(i, key); + metaData.add(i, content); + } + + /** + * Add key/value to file. Keys must be added in an order that agrees with the + * Comparator passed on construction. + * + * @param kv + * KeyValue to add. Cannot be empty nor null. + * @throws IOException + */ + public void append(final KeyValue kv) throws IOException { + append(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(), + kv.getBuffer(), kv.getValueOffset(), kv.getValueLength()); + } + + /** + * Add key/value to file. Keys must be added in an order that agrees with the + * Comparator passed on construction. + * + * @param key + * Key to add. Cannot be empty nor null. + * @param value + * Value to add. Cannot be empty nor null. + * @throws IOException + */ + public void append(final byte[] key, final byte[] value) throws IOException { + append(key, 0, key.length, value, 0, value.length); + } + + /** + * Add key/value to file. Keys must be added in an order that agrees with the + * Comparator passed on construction. + * + * @param key + * @param koffset + * @param klength + * @param value + * @param voffset + * @param vlength + * @throws IOException + */ + private void append(final byte[] key, final int koffset, final int klength, + final byte[] value, final int voffset, final int vlength) + throws IOException { + boolean dupKey = checkKey(key, koffset, klength); + checkValue(value, voffset, vlength); + if (!dupKey) { + checkBlockBoundary(); + } + // Write length of key and value and then actual key and value bytes. + this.out.writeInt(klength); + totalKeyLength += klength; + this.out.writeInt(vlength); + totalValueLength += vlength; + this.out.write(key, koffset, klength); + this.out.write(value, voffset, vlength); + // Are we the first key in this block? + if (this.firstKeyInBlock == null) { + // Copy the key. + this.firstKeyInBlock = new byte[klength]; + System.arraycopy(key, koffset, this.firstKeyInBlock, 0, klength); + } + this.lastKeyBuffer = key; + this.lastKeyOffset = koffset; + this.lastKeyLength = klength; + this.entryCount++; + // If we are pre-caching blocks on write, fill byte array stream + if (cacheDataBlocksOnWrite) { + this.baosDos.writeInt(klength); + this.baosDos.writeInt(vlength); + this.baosDos.write(key, koffset, klength); + this.baosDos.write(value, voffset, vlength); + } + } + + public void close() throws IOException { + if (this.outputStream == null) { + return; + } + // Write out the end of the data blocks, then write meta data blocks. + // followed by fileinfo, data block index and meta block index. + + finishBlock(); + + FixedFileTrailer trailer = new FixedFileTrailer(1); + + // Write out the metadata blocks if any. + ArrayList metaOffsets = null; + ArrayList metaDataSizes = null; + if (metaNames.size() > 0) { + metaOffsets = new ArrayList(metaNames.size()); + metaDataSizes = new ArrayList(metaNames.size()); + for (int i = 0; i < metaNames.size(); ++i) { + // store the beginning offset + long curPos = outputStream.getPos(); + metaOffsets.add(curPos); + // write the metadata content + DataOutputStream dos = getCompressingStream(); + BlockType.META.write(dos); + metaData.get(i).write(dos); + int size = releaseCompressingStream(dos); + // store the metadata size + metaDataSizes.add(size); + } + } + + writeFileInfo(trailer, outputStream); + + // Write the data block index. + trailer.setLoadOnOpenOffset(writeBlockIndex(this.outputStream, + this.blockKeys, this.blockOffsets, this.blockDataSizes)); + LOG.info("Wrote a version 1 block index with " + this.blockKeys.size() + + " keys"); + + if (metaNames.size() > 0) { + // Write the meta index. + writeBlockIndex(this.outputStream, metaNames, metaOffsets, metaDataSizes); + } + + // Now finish off the trailer. + trailer.setDataIndexCount(blockKeys.size()); + + finishClose(trailer); + } + + @Override + protected void finishFileInfo() throws IOException { + super.finishFileInfo(); + + // In version 1, we store comparator name in the file info. + fileInfo.append(FileInfo.COMPARATOR, + Bytes.toBytes(comparator.getClass().getName()), false); + } + + @Override + public void addInlineBlockWriter(InlineBlockWriter bloomWriter) { + // Inline blocks only exist in HFile format version 2. + throw new UnsupportedOperationException(); + } + + /** + * Version 1 Bloom filters are stored in two meta blocks with two different + * keys. + */ + @Override + public void addBloomFilter(BloomFilterWriter bfw) { + appendMetaBlock(BLOOM_FILTER_META_KEY, + bfw.getMetaWriter()); + Writable dataWriter = bfw.getDataWriter(); + if (dataWriter != null) { + appendMetaBlock(BLOOM_FILTER_DATA_KEY, dataWriter); + } + } + + /** + * Write out the index in the version 1 format. This conforms to the legacy + * version 1 format, but can still be read by + * {@link HFileBlockIndex.BlockIndexReader#readRootIndex(java.io.DataInputStream, + * int)}. + * + * @param out the stream to write to + * @param keys + * @param offsets + * @param uncompressedSizes in contrast with a version 2 root index format, + * the sizes stored in the version 1 are uncompressed sizes + * @return + * @throws IOException + */ + private static long writeBlockIndex(final FSDataOutputStream out, + final List keys, final List offsets, + final List uncompressedSizes) throws IOException { + long pos = out.getPos(); + // Don't write an index if nothing in the index. + if (keys.size() > 0) { + BlockType.INDEX_V1.write(out); + // Write the index. + for (int i = 0; i < keys.size(); ++i) { + out.writeLong(offsets.get(i).longValue()); + out.writeInt(uncompressedSizes.get(i).intValue()); + byte[] key = keys.get(i); + Bytes.writeByteArray(out, key); + } + } + return pos; + } + +} \ No newline at end of file diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java new file mode 100644 index 0000000..261ffb9 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java @@ -0,0 +1,452 @@ +/* + * Copyright 2011 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.DataOutput; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.KeyComparator; +import org.apache.hadoop.hbase.io.hfile.HFile.Writer; +import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable; +import org.apache.hadoop.hbase.util.BloomFilterWriter; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Writable; + +/** + * Writes HFile format version 2. + */ +public class HFileWriterV2 extends AbstractHFileWriter { + + /** Inline block writers for multi-level block index and compound Blooms. */ + private List inlineBlockWriters = + new ArrayList(); + + /** Unified version 2 block writer */ + private HFileBlock.Writer fsBlockWriter; + + private HFileBlockIndex.BlockIndexWriter dataBlockIndexWriter; + private HFileBlockIndex.BlockIndexWriter metaBlockIndexWriter; + + /** The offset of the first data block or -1 if the file is empty. */ + private long firstDataBlockOffset = -1; + + /** The offset of the last data block or 0 if the file is empty. */ + private long lastDataBlockOffset; + + /** Additional data items to be written to the "load-on-open" section. */ + private List additionalLoadOnOpenData = + new ArrayList(); + + static class WriterFactoryV2 extends HFile.WriterFactory { + + WriterFactoryV2(Configuration conf) { super(conf); } + + @Override + public Writer createWriter(FileSystem fs, Path path) + throws IOException { + return new HFileWriterV2(conf, fs, path); + } + + @Override + public Writer createWriter(FileSystem fs, Path path, int blockSize, + Compression.Algorithm compress, + final KeyComparator comparator) throws IOException { + return new HFileWriterV2(conf, fs, path, blockSize, + compress, comparator); + } + + @Override + public Writer createWriter(FileSystem fs, Path path, int blockSize, + String compress, final KeyComparator comparator) + throws IOException { + return new HFileWriterV2(conf, fs, path, blockSize, + compress, comparator); + } + + @Override + public Writer createWriter(final FSDataOutputStream ostream, + final int blockSize, final String compress, + final KeyComparator comparator) throws IOException { + return new HFileWriterV2(conf, ostream, blockSize, compress, comparator); + } + + @Override + public Writer createWriter(final FSDataOutputStream ostream, + final int blockSize, final Compression.Algorithm compress, + final KeyComparator c) throws IOException { + return new HFileWriterV2(conf, ostream, blockSize, compress, c); + } + } + + /** Constructor that uses all defaults for compression and block size. */ + public HFileWriterV2(Configuration conf, FileSystem fs, Path path) + throws IOException { + this(conf, fs, path, HFile.DEFAULT_BLOCKSIZE, + HFile.DEFAULT_COMPRESSION_ALGORITHM, null); + } + + /** + * Constructor that takes a path, creates and closes the output stream. Takes + * compression algorithm name as string. + */ + public HFileWriterV2(Configuration conf, FileSystem fs, Path path, + int blockSize, String compressAlgoName, + final KeyComparator comparator) throws IOException { + this(conf, fs, path, blockSize, + compressionByName(compressAlgoName), comparator); + } + + /** Constructor that takes a path, creates and closes the output stream. */ + public HFileWriterV2(Configuration conf, FileSystem fs, Path path, + int blockSize, Compression.Algorithm compressAlgo, + final KeyComparator comparator) throws IOException { + super(conf, createOutputStream(conf, fs, path), path, + blockSize, compressAlgo, comparator); + finishInit(conf); + } + + /** Constructor that takes a stream. */ + public HFileWriterV2(final Configuration conf, + final FSDataOutputStream outputStream, final int blockSize, + final String compressAlgoName, final KeyComparator comparator) + throws IOException { + this(conf, outputStream, blockSize, + Compression.getCompressionAlgorithmByName(compressAlgoName), + comparator); + } + + /** Constructor that takes a stream. */ + public HFileWriterV2(final Configuration conf, + final FSDataOutputStream outputStream, final int blockSize, + final Compression.Algorithm compress, final KeyComparator comparator) + throws IOException { + super(conf, outputStream, null, blockSize, compress, comparator); + finishInit(conf); + } + + /** Additional initialization steps */ + private void finishInit(final Configuration conf) { + if (fsBlockWriter != null) + throw new IllegalStateException("finishInit called twice"); + + // HFile filesystem-level (non-caching) block writer + fsBlockWriter = new HFileBlock.Writer(compressAlgo); + + // Data block index writer + dataBlockIndexWriter = new HFileBlockIndex.BlockIndexWriter(fsBlockWriter, + cacheIndexBlocksOnWrite ? blockCache : null, + cacheIndexBlocksOnWrite ? name : null); + dataBlockIndexWriter.setMaxChunkSize( + HFileBlockIndex.getMaxChunkSize(conf)); + inlineBlockWriters.add(dataBlockIndexWriter); + + // Meta data block index writer + metaBlockIndexWriter = new HFileBlockIndex.BlockIndexWriter(); + } + + /** + * At a block boundary, write all the inline blocks and opens new block. + * + * @throws IOException + */ + private void checkBlockBoundary() throws IOException { + if (fsBlockWriter.blockSizeWritten() < blockSize) + return; + + finishBlock(); + writeInlineBlocks(false); + newBlock(); + } + + /** Clean up the current block */ + private void finishBlock() throws IOException { + if (!fsBlockWriter.isWriting() || fsBlockWriter.blockSizeWritten() == 0) + return; + + long startTimeNs = System.nanoTime(); + + // Update the first data block offset for scanning. + if (firstDataBlockOffset == -1) + firstDataBlockOffset = outputStream.getPos(); + + // Update the last data block offset + lastDataBlockOffset = outputStream.getPos(); + + fsBlockWriter.writeHeaderAndData(outputStream); + + int onDiskSize = fsBlockWriter.getOnDiskSizeWithHeader(); + dataBlockIndexWriter.addEntry(firstKeyInBlock, lastDataBlockOffset, + onDiskSize); + totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader(); + + HFile.writeTimeNano.addAndGet(System.nanoTime() - startTimeNs); + HFile.writeOps.incrementAndGet(); + + if (cacheDataBlocksOnWrite) { + blockCache.cacheBlock(HFile.getBlockCacheKey(name, lastDataBlockOffset), + fsBlockWriter.getBlockForCaching()); + } + } + + /** Gives inline block writers an opportunity to contribute blocks. */ + private void writeInlineBlocks(boolean closing) throws IOException { + for (InlineBlockWriter ibw : inlineBlockWriters) { + while (ibw.shouldWriteBlock(closing)) { + long offset = outputStream.getPos(); + boolean cacheThisBlock = ibw.cacheOnWrite(); + ibw.writeInlineBlock(fsBlockWriter.startWriting( + ibw.getInlineBlockType(), cacheThisBlock)); + fsBlockWriter.writeHeaderAndData(outputStream); + ibw.blockWritten(offset, fsBlockWriter.getOnDiskSizeWithHeader(), + fsBlockWriter.getUncompressedSizeWithoutHeader()); + + if (cacheThisBlock) { + // Cache this block on write. + blockCache.cacheBlock(HFile.getBlockCacheKey(name, offset), + fsBlockWriter.getBlockForCaching()); + } + } + } + } + + /** + * Ready a new block for writing. + * + * @throws IOException + */ + private void newBlock() throws IOException { + // This is where the next block begins. + fsBlockWriter.startWriting(BlockType.DATA, cacheDataBlocksOnWrite); + firstKeyInBlock = null; + } + + /** + * Add a meta block to the end of the file. Call before close(). Metadata + * blocks are expensive. Fill one with a bunch of serialized data rather than + * do a metadata block per metadata instance. If metadata is small, consider + * adding to file info using {@link #appendFileInfo(byte[], byte[])} + * + * @param metaBlockName + * name of the block + * @param content + * will call readFields to get data later (DO NOT REUSE) + */ + @Override + public void appendMetaBlock(String metaBlockName, Writable content) { + byte[] key = Bytes.toBytes(metaBlockName); + int i; + for (i = 0; i < metaNames.size(); ++i) { + // stop when the current key is greater than our own + byte[] cur = metaNames.get(i); + if (Bytes.BYTES_RAWCOMPARATOR.compare(cur, 0, cur.length, key, 0, + key.length) > 0) { + break; + } + } + metaNames.add(i, key); + metaData.add(i, content); + } + + /** + * Add key/value to file. Keys must be added in an order that agrees with the + * Comparator passed on construction. + * + * @param kv + * KeyValue to add. Cannot be empty nor null. + * @throws IOException + */ + @Override + public void append(final KeyValue kv) throws IOException { + append(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(), + kv.getBuffer(), kv.getValueOffset(), kv.getValueLength()); + } + + /** + * Add key/value to file. Keys must be added in an order that agrees with the + * Comparator passed on construction. + * + * @param key + * Key to add. Cannot be empty nor null. + * @param value + * Value to add. Cannot be empty nor null. + * @throws IOException + */ + @Override + public void append(final byte[] key, final byte[] value) throws IOException { + append(key, 0, key.length, value, 0, value.length); + } + + /** + * Add key/value to file. Keys must be added in an order that agrees with the + * Comparator passed on construction. + * + * @param key + * @param koffset + * @param klength + * @param value + * @param voffset + * @param vlength + * @throws IOException + */ + private void append(final byte[] key, final int koffset, final int klength, + final byte[] value, final int voffset, final int vlength) + throws IOException { + boolean dupKey = checkKey(key, koffset, klength); + checkValue(value, voffset, vlength); + if (!dupKey) { + checkBlockBoundary(); + } + + if (!fsBlockWriter.isWriting()) + newBlock(); + + // Write length of key and value and then actual key and value bytes. + { + DataOutputStream out = fsBlockWriter.getUserDataStream(); + out.writeInt(klength); + totalKeyLength += klength; + out.writeInt(vlength); + totalValueLength += vlength; + out.write(key, koffset, klength); + out.write(value, voffset, vlength); + } + + // Are we the first key in this block? + if (firstKeyInBlock == null) { + // Copy the key. + firstKeyInBlock = new byte[klength]; + System.arraycopy(key, koffset, firstKeyInBlock, 0, klength); + } + + lastKeyBuffer = key; + lastKeyOffset = koffset; + lastKeyLength = klength; + entryCount++; + } + + @Override + public void close() throws IOException { + if (outputStream == null) { + return; + } + // Write out the end of the data blocks, then write meta data blocks. + // followed by fileinfo, data block index and meta block index. + + finishBlock(); + writeInlineBlocks(true); + + FixedFileTrailer trailer = new FixedFileTrailer(2); + + // Write out the metadata blocks if any. + if (!metaNames.isEmpty()) { + for (int i = 0; i < metaNames.size(); ++i) { + // store the beginning offset + long offset = outputStream.getPos(); + // write the metadata content + DataOutputStream dos = fsBlockWriter.startWriting(BlockType.META, + cacheDataBlocksOnWrite); + metaData.get(i).write(dos); + + fsBlockWriter.writeHeaderAndData(outputStream); + + // Add the new meta block to the meta index. + metaBlockIndexWriter.addEntry(metaNames.get(i), offset, + fsBlockWriter.getOnDiskSizeWithHeader()); + } + } + + // Load-on-open section. + + // Data block index. + // + // In version 2, this section of the file starts with the root level data + // block index. We call a function that writes intermediate-level blocks + // first, then root level, and returns the offset of the root level block + // index. + + long rootIndexOffset = dataBlockIndexWriter.writeIndexBlocks(outputStream); + trailer.setLoadOnOpenOffset(rootIndexOffset); + + // Meta block index. + metaBlockIndexWriter.writeSingleLevelIndex(fsBlockWriter.startWriting( + BlockType.ROOT_INDEX, false), "meta"); + fsBlockWriter.writeHeaderAndData(outputStream); + + // File info + writeFileInfo(trailer, fsBlockWriter.startWriting(BlockType.FILE_INFO, + false)); + fsBlockWriter.writeHeaderAndData(outputStream); + + // Load-on-open data supplied by higher levels, e.g. Bloom filters. + for (BlockWritable w : additionalLoadOnOpenData) + fsBlockWriter.writeBlock(w, outputStream); + + // Now finish off the trailer. + trailer.setNumDataIndexLevels(dataBlockIndexWriter.getNumLevels()); + trailer.setUncompressedDataIndexSize( + dataBlockIndexWriter.getTotalUncompressedSize()); + trailer.setFirstDataBlockOffset(firstDataBlockOffset); + trailer.setLastDataBlockOffset(lastDataBlockOffset); + trailer.setComparatorClass(comparator.getClass()); + trailer.setDataIndexCount(dataBlockIndexWriter.getNumRootEntries()); + + finishClose(trailer); + + fsBlockWriter.releaseCompressor(); + } + + @Override + public void addInlineBlockWriter(InlineBlockWriter ibw) { + inlineBlockWriters.add(ibw); + if (blockCache == null && ibw.cacheOnWrite()) + initBlockCache(); + } + + @Override + public void addBloomFilter(final BloomFilterWriter bfw) { + if (bfw.getKeyCount() <= 0) + return; + + additionalLoadOnOpenData.add(new BlockWritable() { + @Override + public BlockType getBlockType() { + return BlockType.BLOOM_META; + } + + @Override + public void writeToBlock(DataOutput out) throws IOException { + bfw.getMetaWriter().write(out); + Writable dataWriter = bfw.getDataWriter(); + if (dataWriter != null) + dataWriter.write(out); + } + }); + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/InlineBlockWriter.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/InlineBlockWriter.java new file mode 100644 index 0000000..b43ab67 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/InlineBlockWriter.java @@ -0,0 +1,73 @@ +/* + * Copyright 2011 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.DataOutput; +import java.io.IOException; + +/** + * A way to write "inline" blocks into an {@link HFile}. Inline blocks are + * interspersed with data blocks. For example, Bloom filter chunks and + * leaf-level blocks of a multi-level block index are stored as inline blocks. + */ +public interface InlineBlockWriter { + + /** + * Determines whether there is a new block to be written out. + * + * @param closing + * whether the file is being closed, in which case we need to write + * out all available data and not wait to accumulate another block + */ + boolean shouldWriteBlock(boolean closing); + + /** + * Writes the block to the provided stream. Must not write any magic records. + * Called only if {@link #shouldWriteBlock(boolean)} returned true. + * + * @param out + * a stream (usually a compressing stream) to write the block to + */ + void writeInlineBlock(DataOutput out) throws IOException; + + /** + * Called after a block has been written, and its offset, raw size, and + * compressed size have been determined. Can be used to add an entry to a + * block index. If this type of inline blocks needs a block index, the inline + * block writer is responsible for maintaining it. + * + * @param offset the offset of the block in the stream + * @param onDiskSize the on-disk size of the block + * @param uncompressedSize the uncompressed size of the block + * @param rawSize + */ + void blockWritten(long offset, int onDiskSize, int uncompressedSize); + + /** + * The type of blocks this block writer produces. + */ + BlockType getInlineBlockType(); + + /** + * @return true if inline blocks produced by this writer should be cached + */ + boolean cacheOnWrite(); + +} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java index ecab7ca..88aa652 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.io.hfile; import java.lang.ref.WeakReference; -import java.nio.ByteBuffer; import java.util.LinkedList; import java.util.PriorityQueue; import java.util.concurrent.atomic.AtomicLong; @@ -251,7 +250,7 @@ public class LruBlockCache implements BlockCache, HeapSize { * @param buf block buffer * @param inMemory if block is in-memory */ - public void cacheBlock(String blockName, ByteBuffer buf, boolean inMemory) { + public void cacheBlock(String blockName, HeapSize buf, boolean inMemory) { CachedBlock cb = map.get(blockName); if(cb != null) { throw new RuntimeException("Cached an already cached block"); @@ -275,7 +274,7 @@ public class LruBlockCache implements BlockCache, HeapSize { * @param blockName block name * @param buf block buffer */ - public void cacheBlock(String blockName, ByteBuffer buf) { + public void cacheBlock(String blockName, HeapSize buf) { cacheBlock(blockName, buf, false); } @@ -284,7 +283,7 @@ public class LruBlockCache implements BlockCache, HeapSize { * @param blockName block name * @return buffer of specified block name, or null if not in cache */ - public ByteBuffer getBlock(String blockName, boolean caching) { + public HeapSize getBlock(String blockName, boolean caching) { CachedBlock cb = map.get(blockName); if(cb == null) { stats.miss(caching); @@ -304,6 +303,31 @@ public class LruBlockCache implements BlockCache, HeapSize { return true; } + /** + * Evicts all blocks whose name starts with the given prefix. This is an + * expensive operation implemented as a linear-time search through all blocks + * in the cache. Ideally this should be a search in a log-access-time map. + * + *

+ * This is used for evict-on-close to remove all blocks of a specific HFile. + * The prefix would be the HFile/StoreFile name (a UUID) followed by an + * underscore, because HFile v2 block names in cache are of the form + * "<storeFileUUID>_<blockOffset>". + * + * @return the number of blocks evicted + */ + @Override + public int evictBlocksByPrefix(String prefix) { + int numEvicted = 0; + for (String key : map.keySet()) { + if (key.startsWith(prefix)) { + if (evictBlock(key)) + ++numEvicted; + } + } + return numEvicted; + } + protected long evictBlock(CachedBlock block) { map.remove(block.getName()); size.addAndGet(-1 * block.heapSize()); diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java index 150f54f..886c31d 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java @@ -25,6 +25,7 @@ import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; +import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.LruBlockCache.CacheStats; @@ -32,10 +33,10 @@ import org.apache.hadoop.hbase.io.hfile.LruBlockCache.CacheStats; * Simple one RFile soft reference cache. */ public class SimpleBlockCache implements BlockCache { - private static class Ref extends SoftReference { + private static class Ref extends SoftReference { public String blockId; - public Ref(String blockId, ByteBuffer buf, ReferenceQueue q) { - super(buf, q); + public Ref(String blockId, HeapSize block, ReferenceQueue q) { + super(block, q); this.blockId = blockId; } } @@ -68,7 +69,7 @@ public class SimpleBlockCache implements BlockCache { return cache.size(); } - public synchronized ByteBuffer getBlock(String blockName, boolean caching) { + public synchronized HeapSize getBlock(String blockName, boolean caching) { processQueue(); // clear out some crap. Ref ref = cache.get(blockName); if (ref == null) @@ -76,13 +77,13 @@ public class SimpleBlockCache implements BlockCache { return ref.get(); } - public synchronized void cacheBlock(String blockName, ByteBuffer buf) { - cache.put(blockName, new Ref(blockName, buf, q)); + public synchronized void cacheBlock(String blockName, HeapSize block) { + cache.put(blockName, new Ref(blockName, block, q)); } - public synchronized void cacheBlock(String blockName, ByteBuffer buf, + public synchronized void cacheBlock(String blockName, HeapSize block, boolean inMemory) { - cache.put(blockName, new Ref(blockName, buf, q)); + cache.put(blockName, new Ref(blockName, block, q)); } @Override @@ -117,4 +118,10 @@ public class SimpleBlockCache implements BlockCache { // TODO: implement this if we ever actually use this block cache return 0; } + + @Override + public int evictBlocksByPrefix(String string) { + throw new UnsupportedOperationException(); + } } + diff --git a/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java b/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java index ec38c7d..0fbb471 100644 --- a/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java +++ b/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java @@ -79,7 +79,7 @@ public class HFileOutputFormat extends FileOutputFormat e : this.onlineRegions.entrySet()) { HRegion r = e.getValue(); memstoreSize += r.memstoreSize.get(); @@ -1208,20 +1223,24 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler, Store store = ee.getValue(); storefiles += store.getStorefilesCount(); storefileIndexSize += store.getStorefilesIndexSize(); + totalStaticIndexSize += store.getTotalStaticIndexSize(); + totalStaticBloomSize += store.getTotalStaticBloomSize(); } } } this.metrics.stores.set(stores); this.metrics.storefiles.set(storefiles); this.metrics.memstoreSizeMB.set((int) (memstoreSize / (1024 * 1024))); + this.metrics.storefileIndexSizeMB.set( + (int) (storefileIndexSize / (1024 * 1024))); + this.metrics.rootIndexSizeKB.set( + (int) (storefileIndexSize / 1024)); + this.metrics.totalStaticIndexSizeKB.set( + (int) (totalStaticIndexSize / 1024)); + this.metrics.totalStaticBloomSizeKB.set( + (int) (totalStaticBloomSize / 1024)); this.metrics.readRequestsCount.set(readRequestsCount); this.metrics.writeRequestsCount.set(writeRequestsCount); - this.metrics.storefileIndexSizeMB - .set((int) (storefileIndexSize / (1024 * 1024))); - this.metrics.compactionQueueSize.set(compactSplitThread - .getCompactionQueueSize()); - this.metrics.flushQueueSize.set(cacheFlusher - .getFlushQueueSize()); BlockCache blockCache = StoreFile.getBlockCache(conf); if (blockCache != null) { diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index 97385c8..5aedc47 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -24,10 +24,8 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.NavigableSet; -import java.util.Set; import java.util.SortedSet; import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -59,7 +57,6 @@ import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; /** * A Store holds a column family in a Region. Its a memstore and a set of zero @@ -342,7 +339,7 @@ public class Store implements HeapSize { try { LOG.info("Validating hfile at " + srcPath + " for inclusion in " + "store " + this + " region " + this.region); - reader = new HFile.Reader(srcPath.getFileSystem(conf), + reader = HFile.createReader(srcPath.getFileSystem(conf), srcPath, null, false, false); reader.loadFileInfo(); @@ -556,8 +553,7 @@ public class Store implements HeapSize { throws IOException { return StoreFile.createWriter(this.fs, region.getTmpDir(), this.blocksize, compression, this.comparator, this.conf, - this.family.getBloomFilterType(), maxKeyCount, - conf.getBoolean("hbase.rs.cacheblocksonwrite", false)); + this.family.getBloomFilterType(), maxKeyCount); } /* @@ -1575,6 +1571,37 @@ public class Store implements HeapSize { } /** + * Returns the total size of all index blocks in the data block indexes, + * including the root level, intermediate levels, and the leaf level for + * multi-level indexes, or just the root level for single-level indexes. + * + * @return the total size of block indexes in the store + */ + long getTotalStaticIndexSize() { + long size = 0; + for (StoreFile s : storefiles) { + size += s.getReader().getUncompressedDataIndexSize(); + } + return size; + } + + /** + * Returns the total byte size of all Bloom filter bit arrays. For compound + * Bloom filters even the Bloom blocks currently not loaded into the block + * cache are counted. + * + * @return the total size of all Bloom filters in the store + */ + long getTotalStaticBloomSize() { + long size = 0; + for (StoreFile s : storefiles) { + StoreFile.Reader r = s.getReader(); + size += r.getTotalBloomSize(); + } + return size; + } + + /** * @return The priority that this store should have in the compaction queue */ public int getCompactPriority() { diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 9cc75bb..94c8bb4 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -19,12 +19,12 @@ */ package org.apache.hadoop.hbase.regionserver; +import java.io.DataInput; import java.io.FileNotFoundException; import java.io.IOException; import java.lang.management.ManagementFactory; import java.lang.management.MemoryUsage; import java.nio.ByteBuffer; -import java.text.NumberFormat; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -52,11 +52,12 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileScanner; +import org.apache.hadoop.hbase.io.hfile.HFileWriterV1; import org.apache.hadoop.hbase.io.hfile.LruBlockCache; import org.apache.hadoop.hbase.util.BloomFilter; -import org.apache.hadoop.hbase.util.ByteBloomFilter; +import org.apache.hadoop.hbase.util.BloomFilterFactory; +import org.apache.hadoop.hbase.util.BloomFilterWriter; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Hash; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.WritableUtils; @@ -69,10 +70,10 @@ import com.google.common.collect.Ordering; /** * A Store data file. Stores usually have one or more of these files. They * are produced by flushing the memstore to disk. To - * create, call {@link #createWriter(FileSystem, Path, int)} 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 + * create, call {@link #createWriter(FileSystem, Path, int, Configuration)} + * 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 #createReader()}. *

StoreFiles may also reference store files in another Store. * @@ -82,11 +83,6 @@ import com.google.common.collect.Ordering; public class StoreFile { static final Log LOG = LogFactory.getLog(StoreFile.class.getName()); - // Config keys. - static final String IO_STOREFILE_BLOOM_ERROR_RATE = "io.storefile.bloom.error.rate"; - static final String IO_STOREFILE_BLOOM_MAX_FOLD = "io.storefile.bloom.max.fold"; - static final String IO_STOREFILE_BLOOM_MAX_KEYS = "io.storefile.bloom.max.keys"; - static final String IO_STOREFILE_BLOOM_ENABLED = "io.storefile.bloom.enabled"; static final String HFILE_BLOCK_CACHE_SIZE_KEY = "hfile.block.cache.size"; public static enum BloomType { @@ -103,21 +99,26 @@ public class StoreFile { */ ROWCOL } + // Keys for fileinfo values in HFile + /** Max Sequence ID in FileInfo */ public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY"); + /** Major compaction flag in FileInfo */ - public static final byte [] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY"); + public static final byte[] MAJOR_COMPACTION_KEY = + Bytes.toBytes("MAJOR_COMPACTION_KEY"); + /** Bloom filter Type in FileInfo */ - static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE"); + static final byte[] BLOOM_FILTER_TYPE_KEY = + Bytes.toBytes("BLOOM_FILTER_TYPE"); + + /** Last Bloom filter key in FileInfo */ + private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY"); + /** Key for Timerange information in metadata*/ public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE"); - /** Meta data block name for bloom filter meta-info (ie: bloom params/specs) */ - static final String BLOOM_FILTER_META_KEY = "BLOOM_FILTER_META"; - /** Meta data block name for bloom filter data (ie: bloom bits) */ - static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA"; - // Make default block size for StoreFiles 8k while testing. TODO: FIX! // Need to make it 8k for testing. public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024; @@ -126,14 +127,19 @@ public class StoreFile { private static BlockCache hfileBlockCache = null; 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; + // Should the block cache be used or not. private boolean blockcache; + // Is this from an in-memory store private boolean inMemory; @@ -204,11 +210,12 @@ public class StoreFile { } // ignore if the column family config says "no bloom filter" // even if there is one in the hfile. - if (conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true)) { + if (BloomFilterFactory.isBloomEnabled(conf)) { this.bloomType = bt; } else { + LOG.info("Ignoring bloom filter check for file " + path + ": " + + "bloomType=" + bt + " (disabled in config)"); this.bloomType = BloomType.NONE; - LOG.info("Ignoring bloom filter check for file (disabled in config)"); } // cache the modification time stamp of this store file @@ -393,7 +400,7 @@ public class StoreFile { } else { this.reader = new Reader(this.fs, this.path, getBlockCache(), this.inMemory, - this.conf.getBoolean("hbase.rs.evictblocksonclose", true)); + this.conf.getBoolean(HFile.EVICT_BLOCKS_ON_CLOSE_KEY, true)); } // Load up indices and fileinfo. metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo()); @@ -541,13 +548,10 @@ public class StoreFile { * @return StoreFile.Writer * @throws IOException */ - public static Writer createWriter(final FileSystem fs, - final Path dir, - final int blocksize) - throws IOException { - - return createWriter(fs, dir, blocksize, null, null, null, BloomType.NONE, 0, - false); + public static Writer createWriter(final FileSystem fs, final Path dir, + final int blocksize, Configuration conf) throws IOException { + return createWriter(fs, dir, blocksize, null, null, conf, BloomType.NONE, + 0); } /** @@ -558,10 +562,10 @@ public class StoreFile { * 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. * @param conf HBase system configuration. used with bloom filters * @param bloomType column family setting for bloom filters - * @param c Pass null to get default. - * @param maxKeySize peak theoretical entry size (maintains error rate) + * @param maxKeyCount estimated maximum number of keys we expect to add * @return HFile.Writer * @throws IOException */ @@ -572,22 +576,20 @@ public class StoreFile { final KeyValue.KVComparator c, final Configuration conf, BloomType bloomType, - int maxKeySize, - final boolean cacheOnWrite) + long maxKeyCount) throws IOException { if (!fs.exists(dir)) { fs.mkdirs(dir); } Path path = getUniqueFile(fs, dir); - if(conf == null || !conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true)) { + if (!BloomFilterFactory.isBloomEnabled(conf)) { bloomType = BloomType.NONE; } return new Writer(fs, path, blocksize, algorithm == null? HFile.DEFAULT_COMPRESSION_ALGORITHM: algorithm, - conf, c == null? KeyValue.COMPARATOR: c, bloomType, maxKeySize, - cacheOnWrite); + conf, c == null ? KeyValue.COMPARATOR: c, bloomType, maxKeyCount); } /** @@ -677,11 +679,13 @@ public class StoreFile { * local because it is an implementation detail of the HBase regionserver. */ public static class Writer { - private final BloomFilter bloomFilter; + private final BloomFilterWriter bloomFilterWriter; private final BloomType bloomType; + private byte[] lastBloomKey; + private int lastBloomKeyOffset, lastBloomKeyLen; private KVComparator kvComparator; private KeyValue lastKv = null; - private byte[] lastByteArray = null; + TimeRangeTracker timeRangeTracker = new TimeRangeTracker(); /* isTimeRangeTrackerSet keeps track if the timeRange has already been set * When flushing a memstore, we set TimeRange and use this variable to @@ -701,59 +705,30 @@ public class StoreFile { * @param conf user configuration * @param comparator key comparator * @param bloomType bloom filter setting - * @param maxKeys maximum amount of keys to add (for blooms) - * @param cacheOnWrite whether to cache blocks as we write file + * @param maxKeys the expected maximum number of keys to be added. Was used + * for Bloom filter size in {@link HFile} format version 1. * @throws IOException problem writing to FS */ public Writer(FileSystem fs, Path path, int blocksize, Compression.Algorithm compress, final Configuration conf, - final KVComparator comparator, BloomType bloomType, int maxKeys, - boolean cacheOnWrite) + final KVComparator comparator, BloomType bloomType, long maxKeys) throws IOException { - writer = new HFile.Writer(fs, path, blocksize, compress, - comparator.getRawComparator(), - cacheOnWrite ? StoreFile.getBlockCache(conf) : null); + writer = HFile.getWriterFactory(conf).createWriter( + fs, path, blocksize, + compress, comparator.getRawComparator()); this.kvComparator = comparator; - BloomFilter bloom = null; - BloomType bt = BloomType.NONE; - - if (bloomType != BloomType.NONE && conf != null) { - float err = conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float)0.01); - // Since in row+col blooms we have 2 calls to shouldSeek() instead of 1 - // and the false positives are adding up, we should keep the error rate - // twice as low in order to maintain the number of false positives as - // desired by the user - if (bloomType == BloomType.ROWCOL) { - err /= 2; - } - int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, 7); - int tooBig = conf.getInt(IO_STOREFILE_BLOOM_MAX_KEYS, 128*1000*1000); - - if (maxKeys < tooBig) { - try { - bloom = new ByteBloomFilter(maxKeys, err, - Hash.getHashType(conf), maxFold); - bloom.allocBloom(); - bt = bloomType; - } catch (IllegalArgumentException iae) { - LOG.warn(String.format( - "Parse error while creating bloom for %s (%d, %f)", - path, maxKeys, err), iae); - bloom = null; - bt = BloomType.NONE; - } - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Skipping bloom filter because max keysize too large: " - + maxKeys); - } - } + bloomFilterWriter = BloomFilterFactory.createBloomAtWrite(conf, + bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer); + if (bloomFilterWriter != null) { + this.bloomType = bloomType; + LOG.info("Bloom filter type for " + path + ": " + this.bloomType + + ", "+ bloomFilterWriter.getClass().getSimpleName()); + } else { + // Not using Bloom filters. + this.bloomType = BloomType.NONE; } - - this.bloomFilter = bloom; - this.bloomType = bt; } /** @@ -812,7 +787,7 @@ public class StoreFile { } public void append(final KeyValue kv) throws IOException { - if (this.bloomFilter != null) { + if (this.bloomFilterWriter != null) { // only add to the bloom filter on a new, unique key boolean newKey = true; if (this.lastKv != null) { @@ -836,24 +811,42 @@ public class StoreFile { * 1. Row = Row * 2. RowCol = Row + Qualifier */ + byte[] bloomKey; + int bloomKeyOffset, bloomKeyLen; + switch (bloomType) { case ROW: - this.bloomFilter.add(kv.getBuffer(), kv.getRowOffset(), - kv.getRowLength()); + bloomKey = kv.getBuffer(); + bloomKeyOffset = kv.getRowOffset(); + bloomKeyLen = kv.getRowLength(); break; case ROWCOL: // merge(row, qualifier) - int ro = kv.getRowOffset(); - int rl = kv.getRowLength(); - int qo = kv.getQualifierOffset(); - int ql = kv.getQualifierLength(); - byte [] result = new byte[rl + ql]; - System.arraycopy(kv.getBuffer(), ro, result, 0, rl); - System.arraycopy(kv.getBuffer(), qo, result, rl, ql); - this.bloomFilter.add(result); + // TODO: could save one buffer copy in case of compound Bloom + // filters when this involves creating a KeyValue + bloomKey = bloomFilterWriter.createBloomKey(kv.getBuffer(), + kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(), + kv.getQualifierOffset(), kv.getQualifierLength()); + bloomKeyOffset = 0; + bloomKeyLen = bloomKey.length; break; default: + throw new IOException("Invalid Bloom filter type: " + bloomType); + } + bloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); + if (lastBloomKey != null + && bloomFilterWriter.getComparator().compare(bloomKey, + bloomKeyOffset, bloomKeyLen, lastBloomKey, + lastBloomKeyOffset, lastBloomKeyLen) <= 0) { + throw new IOException("Non-increasing Bloom keys: " + + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + + " after " + + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, + lastBloomKeyLen)); } + lastBloomKey = bloomKey; + lastBloomKeyOffset = bloomKeyOffset; + lastBloomKeyLen = bloomKeyLen; this.lastKv = kv; } } @@ -866,39 +859,41 @@ public class StoreFile { } boolean hasBloom() { - return this.bloomFilter != null; + return this.bloomFilterWriter != null; } - public void append(final byte [] key, final byte [] value) throws IOException { - if (this.bloomFilter != null) { - // only add to the bloom filter on a new row - if (this.lastByteArray == null || !Arrays.equals(key, lastByteArray)) { - this.bloomFilter.add(key); - this.lastByteArray = key; - } - } - writer.append(key, value); - includeInTimeRangeTracker(key); + /** + * For unit testing only. + * @return the Bloom filter used by this writer. + */ + BloomFilterWriter getBloomWriter() { + return bloomFilterWriter; } public void close() throws IOException { - // make sure we wrote something to the bloom before adding it - if (this.bloomFilter != null && this.bloomFilter.getKeyCount() > 0) { - bloomFilter.compactBloom(); - if (this.bloomFilter.getMaxKeys() > 0) { - int b = this.bloomFilter.getByteSize(); - int k = this.bloomFilter.getKeyCount(); - int m = this.bloomFilter.getMaxKeys(); - StoreFile.LOG.info("Bloom added to HFile (" + - getPath() + "): " + StringUtils.humanReadableInt(b) + ", " + - k + "/" + m + " (" + NumberFormat.getPercentInstance().format( - ((double)k) / ((double)m)) + ")"); + // Make sure we wrote something to the Bloom filter before adding it. + boolean haveBloom = bloomFilterWriter != null && + bloomFilterWriter.getKeyCount() > 0; + if (haveBloom) { + bloomFilterWriter.compactBloom(); + writer.addBloomFilter(bloomFilterWriter); + writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, + Bytes.toBytes(bloomType.toString())); + if (lastBloomKey != null) { + writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange( + lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset + + lastBloomKeyLen)); } - writer.appendMetaBlock(BLOOM_FILTER_META_KEY, bloomFilter.getMetaWriter()); - writer.appendMetaBlock(BLOOM_FILTER_DATA_KEY, bloomFilter.getDataWriter()); - writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString())); } writer.close(); + + // Log final Bloom filter statistics. This needs to be done after close() + // because compound Bloom filters might be finalized as part of closing. + if (haveBloom && bloomFilterWriter.getMaxKeys() > 0) { + StoreFile.LOG.info("Bloom added to HFile (" + + getPath() + "): " + + bloomFilterWriter.toString().replace("\n", "; ")); + } } public void appendFileInfo(byte[] key, byte[] value) throws IOException { @@ -917,11 +912,12 @@ public class StoreFile { private final HFile.Reader reader; protected TimeRangeTracker timeRangeTracker = null; protected long sequenceID = -1; + private byte[] lastBloomKey; public Reader(FileSystem fs, Path path, BlockCache blockCache, boolean inMemory, boolean evictOnClose) throws IOException { - reader = new HFile.Reader(fs, path, blockCache, inMemory, evictOnClose); + reader = HFile.createReader(fs, path, blockCache, inMemory, evictOnClose); bloomFilterType = BloomType.NONE; } @@ -966,7 +962,7 @@ public class StoreFile { } public boolean shouldSeek(Scan scan, final SortedSet columns) { - return (passesTimerangeFilter(scan) && passesBloomFilter(scan,columns)); + return (passesTimerangeFilter(scan) && passesBloomFilter(scan, columns)); } /** @@ -982,42 +978,82 @@ public class StoreFile { } } - private boolean passesBloomFilter(Scan scan, final SortedSet columns) { - BloomFilter bm = this.bloomFilter; - if (bm == null || !scan.isGetScan()) { + private boolean passesBloomFilter(Scan scan, + final SortedSet columns) { + BloomFilter bloomFilter = this.bloomFilter; + if (bloomFilter == null) { return true; } + + // Empty file? + if (reader.getTrailer().getEntryCount() == 0) + return false; + byte[] row = scan.getStartRow(); byte[] key; switch (this.bloomFilterType) { case ROW: key = row; break; + case ROWCOL: if (columns != null && columns.size() == 1) { - byte[] col = columns.first(); - key = Bytes.add(row, col); + byte[] column = columns.first(); + key = bloomFilter.createBloomKey(row, 0, row.length, + column, 0, column.length); break; } - //$FALL-THROUGH$ + return true; + default: return true; } try { - ByteBuffer bloom = reader.getMetaBlock(BLOOM_FILTER_DATA_KEY, true); - if (bloom != null) { - if (this.bloomFilterType == BloomType.ROWCOL) { + boolean shouldCheckBloom; + ByteBuffer bloom; + if (bloomFilter.supportsAutoLoading()) { + bloom = null; + shouldCheckBloom = true; + } else { + bloom = reader.getMetaBlock(HFileWriterV1.BLOOM_FILTER_DATA_KEY, + true); + shouldCheckBloom = bloom != null; + } + + if (shouldCheckBloom) { + boolean exists; + + // Whether the primary Bloom key is greater than the last Bloom key + // from the file info. For row-column Bloom filters this is not yet + // a sufficient condition to return false. + boolean keyIsAfterLast = lastBloomKey != null + && bloomFilter.getComparator().compare(key, lastBloomKey) > 0; + + if (bloomFilterType == BloomType.ROWCOL) { // Since a Row Delete is essentially a DeleteFamily applied to all // columns, a file might be skipped if using row+col Bloom filter. // In order to ensure this file is included an additional check is // required looking only for a row bloom. - return bm.contains(key, bloom) || - bm.contains(row, bloom); - } - else { - return bm.contains(key, bloom); + byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length, + null, 0, 0); + + if (keyIsAfterLast + && bloomFilter.getComparator().compare(rowBloomKey, + lastBloomKey) > 0) { + exists = false; + } else { + exists = + this.bloomFilter.contains(key, 0, key.length, bloom) || + this.bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length, + bloom); + } + } else { + exists = !keyIsAfterLast + && this.bloomFilter.contains(key, 0, key.length, bloom); } + + return exists; } } catch (IOException e) { LOG.error("Error reading bloom filter data -- proceeding without", @@ -1039,6 +1075,8 @@ public class StoreFile { bloomFilterType = BloomType.valueOf(Bytes.toString(b)); } + lastBloomKey = fi.get(LAST_BLOOM_KEY); + return fi; } @@ -1048,16 +1086,17 @@ public class StoreFile { } try { - ByteBuffer b = reader.getMetaBlock(BLOOM_FILTER_META_KEY, false); - if (b != null) { + DataInput bloomMeta = reader.getBloomFilterMetadata(); + if (bloomMeta != null) { if (bloomFilterType == BloomType.NONE) { - throw new IOException("valid bloom filter type not found in FileInfo"); + throw new IOException( + "valid bloom filter type not found in FileInfo"); } - - this.bloomFilter = new ByteBloomFilter(b); - LOG.info("Loaded " + (bloomFilterType== BloomType.ROW? "row":"col") - + " bloom filter metadata for " + reader.getName()); + bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader); + LOG.info("Loaded " + bloomFilterType + " " + + bloomFilter.getClass().getSimpleName() + " metadata for " + + reader.getName()); } } catch (IOException e) { LOG.error("Error reading bloom filter meta -- proceeding without", e); @@ -1068,13 +1107,16 @@ public class StoreFile { } } - public int getFilterEntries() { - return (this.bloomFilter != null) ? this.bloomFilter.getKeyCount() - : reader.getFilterEntries(); - } - - public ByteBuffer getMetaBlock(String bloomFilterDataKey, boolean cacheBlock) throws IOException { - return reader.getMetaBlock(bloomFilterDataKey, cacheBlock); + /** + * The number of Bloom filter entries in this store file, or an estimate + * thereof, if the Bloom filter is not loaded. This always returns an upper + * bound of the number of Bloom filter entries. + * + * @return an estimate of the number of Bloom filter entries in this file + */ + public long getFilterEntries() { + return bloomFilter != null ? bloomFilter.getKeyCount() + : reader.getEntries(); } public void setBloomFilterFaulty() { @@ -1094,10 +1136,10 @@ public class StoreFile { } public long getTotalUncompressedBytes() { - return reader.getTotalUncompressedBytes(); + return reader.getTrailer().getTotalUncompressedBytes(); } - public int getEntries() { + public long getEntries() { return reader.getEntries(); } @@ -1120,6 +1162,28 @@ public class StoreFile { public void setSequenceID(long sequenceID) { this.sequenceID = sequenceID; } + + BloomFilter getBloomFilter() { + return bloomFilter; + } + + long getUncompressedDataIndexSize() { + return reader.getTrailer().getUncompressedDataIndexSize(); + } + + public long getTotalBloomSize() { + if (bloomFilter == null) + return 0; + return bloomFilter.getByteSize(); + } + + public int getHFileVersion() { + return reader.getTrailer().getVersion(); + } + + HFile.Reader getHFileReader() { + return reader; + } } /** @@ -1171,4 +1235,5 @@ public class StoreFile { } }); } + } diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java b/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java index cc2b2b4..c27f181 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java @@ -138,11 +138,22 @@ public class RegionServerMetrics implements Updater { public final MetricsLongValue writeRequestsCount = new MetricsLongValue("writeRequestsCount", registry); /** - * Sum of all the storefile index sizes in this regionserver in MB */ public final MetricsIntValue storefileIndexSizeMB = new MetricsIntValue("storefileIndexSizeMB", registry); + /** The total size of block index root levels in this regionserver in KB. */ + public final MetricsIntValue rootIndexSizeKB = + new MetricsIntValue("rootIndexSizeKB", registry); + + /** Total size of all block indexes (not necessarily loaded in memory) */ + public final MetricsIntValue totalStaticIndexSizeKB = + new MetricsIntValue("totalStaticIndexSizeKB", registry); + + /** Total size of all Bloom filters (not necessarily loaded in memory) */ + public final MetricsIntValue totalStaticBloomSizeKB = + new MetricsIntValue("totalStaticBloomSizeKB", registry); + /** * Sum of all the memstore sizes in this regionserver in MB */ @@ -252,6 +263,9 @@ public class RegionServerMetrics implements Updater { this.stores.pushMetric(this.metricsRecord); this.storefiles.pushMetric(this.metricsRecord); this.storefileIndexSizeMB.pushMetric(this.metricsRecord); + this.rootIndexSizeKB.pushMetric(this.metricsRecord); + this.totalStaticIndexSizeKB.pushMetric(this.metricsRecord); + this.totalStaticBloomSizeKB.pushMetric(this.metricsRecord); this.memstoreSizeMB.pushMetric(this.metricsRecord); this.readRequestsCount.pushMetric(this.metricsRecord); this.writeRequestsCount.pushMetric(this.metricsRecord); @@ -278,9 +292,9 @@ public class RegionServerMetrics implements Updater { // } // Means you can't pass a numOps of zero or get a ArithmeticException / by zero. int ops = (int)HFile.getReadOps(); - if (ops != 0) this.fsReadLatency.inc(ops, HFile.getReadTime()); + if (ops != 0) this.fsReadLatency.inc(ops, HFile.getReadTimeMs()); ops = (int)HFile.getWriteOps(); - if (ops != 0) this.fsWriteLatency.inc(ops, HFile.getWriteTime()); + if (ops != 0) this.fsWriteLatency.inc(ops, HFile.getWriteTimeMs()); // mix in HLog metrics ops = (int)HLog.getWriteOps(); if (ops != 0) this.fsWriteLatency.inc(ops, HLog.getWriteTime()); @@ -356,6 +370,12 @@ public class RegionServerMetrics implements Updater { Integer.valueOf(this.storefiles.get())); sb = Strings.appendKeyValue(sb, "storefileIndexSize", Integer.valueOf(this.storefileIndexSizeMB.get())); + sb = Strings.appendKeyValue(sb, "rootIndexSizeKB", + Integer.valueOf(this.rootIndexSizeKB.get())); + sb = Strings.appendKeyValue(sb, "totalStaticIndexSizeKB", + Integer.valueOf(this.totalStaticIndexSizeKB.get())); + sb = Strings.appendKeyValue(sb, "totalStaticBloomSizeKB", + Integer.valueOf(this.totalStaticBloomSizeKB.get())); sb = Strings.appendKeyValue(sb, "memstoreSize", Integer.valueOf(this.memstoreSizeMB.get())); sb = Strings.appendKeyValue(sb, "readRequestsCount", diff --git a/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java b/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java index f100366..42378f5 100644 --- a/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java +++ b/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java @@ -19,61 +19,28 @@ */ package org.apache.hadoop.hbase.util; -import org.apache.hadoop.io.Writable; - import java.nio.ByteBuffer; /** * Defines the general behavior of a bloom filter. - *

- * The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by - * the networking research community in the past decade thanks to the bandwidth efficiencies that it - * offers for the transmission of set membership information between networked hosts. A sender encodes - * the information into a bit vector, the Bloom filter, that is more compact than a conventional - * representation. Computation and space costs for construction are linear in the number of elements. - * The receiver uses the filter to test whether various elements are members of the set. Though the - * filter will occasionally return a false positive, it will never return a false negative. When creating - * the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size. * *

- * Originally created by - * European Commission One-Lab Project 034819. + * The Bloom filter is a data structure that was introduced in 1970 and that + * has been adopted by the networking research community in the past decade + * thanks to the bandwidth efficiencies that it offers for the transmission of + * set membership information between networked hosts. A sender encodes the + * information into a bit vector, the Bloom filter, that is more compact than a + * conventional representation. Computation and space costs for construction + * are linear in the number of elements. The receiver uses the filter to test + * whether various elements are members of the set. Though the filter will + * occasionally return a false positive, it will never return a false negative. + * When creating the filter, the sender can choose its desired point in a + * trade-off between the false positive rate and the size. * - *

- * It must be extended in order to define the real behavior. + * @see {@link BloomFilterWriter} for the ability to add elements to a Bloom + * filter */ -public interface BloomFilter { - /** - * Allocate memory for the bloom filter data. Note that bloom data isn't - * allocated by default because it can grow large & reads would be better - * managed by the LRU cache. - */ - void allocBloom(); - - /** - * Add the specified binary to the bloom filter. - * - * @param buf data to be added to the bloom - */ - void add(byte []buf); - - /** - * Add the specified binary to the bloom filter. - * - * @param buf data to be added to the bloom - * @param offset offset into the data to be added - * @param len length of the data to be added - */ - void add(byte []buf, int offset, int len); - - /** - * Check if the specified key is contained in the bloom filter. - * - * @param buf data to check for existence of - * @param bloom bloom filter data to search - * @return true if matched by bloom, false if not - */ - boolean contains(byte [] buf, ByteBuffer bloom); +public interface BloomFilter extends BloomFilterBase { /** * Check if the specified key is contained in the bloom filter. @@ -81,41 +48,16 @@ public interface BloomFilter { * @param buf data to check for existence of * @param offset offset into the data * @param length length of the data - * @param bloom bloom filter data to search + * @param bloom bloom filter data to search. This can be null if auto-loading + * is supported. * @return true if matched by bloom, false if not */ boolean contains(byte [] buf, int offset, int length, ByteBuffer bloom); /** - * @return The number of keys added to the bloom - */ - int getKeyCount(); - - /** - * @return The max number of keys that can be inserted - * to maintain the desired error rate - */ - public int getMaxKeys(); - - /** - * @return Size of the bloom, in bytes + * @return true if this Bloom filter can automatically load its data + * and thus allows a null byte buffer to be passed to contains() */ - public int getByteSize(); + boolean supportsAutoLoading(); - /** - * Compact the bloom before writing metadata & data to disk - */ - void compactBloom(); - - /** - * Get a writable interface into bloom filter meta data. - * @return writable class - */ - Writable getMetaWriter(); - - /** - * Get a writable interface into bloom filter data (actual bloom). - * @return writable class - */ - Writable getDataWriter(); } diff --git a/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java b/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java new file mode 100644 index 0000000..ab218b7 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java @@ -0,0 +1,56 @@ +/* + * Copyright 2011 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.util; + +import org.apache.hadoop.io.RawComparator; + +/** + * Common methods Bloom filter methods required at read and write time. + */ +public interface BloomFilterBase { + + /** + * @return The number of keys added to the bloom + */ + long getKeyCount(); + + /** + * @return The max number of keys that can be inserted + * to maintain the desired error rate + */ + long getMaxKeys(); + + /** + * @return Size of the bloom, in bytes + */ + long getByteSize(); + + /** + * Create a key for a row-column Bloom filter. + */ + byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen, + byte[] qualBuf, int qualOffset, int qualLen); + + /** + * @return Bloom key comparator + */ + RawComparator getComparator(); + +} diff --git a/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java b/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java new file mode 100644 index 0000000..b31121b --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java @@ -0,0 +1,208 @@ +/* + * Copyright 2011 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.util; + +import java.io.DataInput; +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType; + +/** + * Handles Bloom filter initialization based on configuration and serialized + * metadata in the reader and writer of {@link StoreFile}. + */ +public final class BloomFilterFactory { + + private static final Log LOG = + LogFactory.getLog(BloomFilterFactory.class.getName()); + + /** This class should not be instantiated. */ + private BloomFilterFactory() {} + + /** + * Specifies the target error rate to use when selecting the number of keys + * per Bloom filter. + */ + public static final String IO_STOREFILE_BLOOM_ERROR_RATE = + "io.storefile.bloom.error.rate"; + + /** + * Maximum folding factor allowed. The Bloom filter will be shrunk by + * the factor of up to 2 ** this times if we oversize it initially. + */ + public static final String IO_STOREFILE_BLOOM_MAX_FOLD = + "io.storefile.bloom.max.fold"; + + /** + * For default (single-block) Bloom filters this specifies the maximum number + * of keys. + */ + public static final String IO_STOREFILE_BLOOM_MAX_KEYS = + "io.storefile.bloom.max.keys"; + + /** Master switch to enable Bloom filters */ + public static final String IO_STOREFILE_BLOOM_ENABLED = + "io.storefile.bloom.enabled"; + + /** + * Target Bloom block size. Bloom filter blocks of approximately this size + * are interleaved with data blocks. + */ + public static final String IO_STOREFILE_BLOOM_BLOCK_SIZE = + "io.storefile.bloom.block.size"; + + /** Whether to cache compound Bloom filter blocks on write */ + public static final String IO_STOREFILE_BLOOM_CACHE_ON_WRITE = + "io.storefile.bloom.cacheonwrite"; + + /** Maximum number of times a Bloom filter can be "folded" if oversized */ + private static final int MAX_ALLOWED_FOLD_FACTOR = 7; + + /** + * Instantiates the correct Bloom filter class based on the version provided + * in the meta block data. + * + * @param meta the byte array holding the Bloom filter's metadata, including + * version information + * @param reader the {@link HFile} reader to use to lazily load Bloom filter + * blocks + * @return an instance of the correct type of Bloom filter + * @throws IllegalArgumentException + */ + public static BloomFilter + createFromMeta(DataInput meta, HFile.Reader reader) + throws IllegalArgumentException, IOException { + int version = meta.readInt(); + switch (version) { + case ByteBloomFilter.VERSION: + // This is only possible in a version 1 HFile. We are ignoring the + // passed comparator because raw byte comparators are always used + // in version 1 Bloom filters. + return new ByteBloomFilter(meta); + + case CompoundBloomFilterBase.VERSION: + return new CompoundBloomFilter(meta, reader); + + default: + throw new IllegalArgumentException( + "Bad bloom filter format version " + version + ); + } + } + + /** + * @return true if Bloom filters are enabled in the given configuration + */ + public static boolean isBloomEnabled(Configuration conf) { + return conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true); + } + + public static float getErrorRate(Configuration conf) { + return conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float) 0.01); + } + + /** + * Creates a new Bloom filter at the time of + * {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing. + * + * @param conf + * @param bloomType + * @param maxKeys an estimate of the number of keys we expect to insert. + * Irrelevant if compound Bloom filters are enabled. + * @param writer the HFile writer + * @param comparator the comparator to use for compound Bloom filters. This + * has no effect if creating single-chunk version 1 Bloom filters. + * @return the new Bloom filter, or null in case Bloom filters are disabled + * or when failed to create one. + */ + public static BloomFilterWriter createBloomAtWrite(Configuration conf, + BloomType bloomType, int maxKeys, HFile.Writer writer) { + if (!isBloomEnabled(conf)) { + LOG.info("Bloom filters are disabled by configuration for " + + writer.getPath() + + (conf == null ? " (configuration is null)" : "")); + return null; + } else if (bloomType == BloomType.NONE) { + LOG.info("Bloom filter is turned off for the column family"); + return null; + } + + float err = getErrorRate(conf); + + // In case of row/column Bloom filter lookups, each lookup is an OR if two + // separate lookups. Therefore, if each lookup's false positive rate is p, + // the resulting false positive rate is err = 1 - (1 - p)^2, and + // p = 1 - sqrt(1 - err). + if (bloomType == BloomType.ROWCOL) { + err = (float) (1 - Math.sqrt(1 - err)); + } + + int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, + MAX_ALLOWED_FOLD_FACTOR); + + if (HFile.getFormatVersion(conf) > HFile.MIN_FORMAT_VERSION) { + // In case of compound Bloom filters we ignore the maxKeys hint. + CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter( + getBloomBlockSize(conf), err, Hash.getHashType(conf), maxFold, + cacheChunksOnWrite(conf), bloomType == BloomType.ROWCOL + ? KeyValue.KEY_COMPARATOR : Bytes.BYTES_RAWCOMPARATOR); + writer.addInlineBlockWriter(bloomWriter); + return bloomWriter; + } else { + // A single-block Bloom filter. Only used when testing HFile format + // version 1. + int tooBig = conf.getInt(IO_STOREFILE_BLOOM_MAX_KEYS, + 128 * 1000 * 1000); + + if (maxKeys <= 0) { + LOG.warn("Invalid maximum number of keys specified: " + maxKeys + + ", not using Bloom filter"); + return null; + } else if (maxKeys < tooBig) { + BloomFilterWriter bloom = new ByteBloomFilter((int) maxKeys, err, + Hash.getHashType(conf), maxFold); + bloom.allocBloom(); + return bloom; + } else { + LOG.debug("Skipping bloom filter because max keysize too large: " + + maxKeys); + } + } + return null; + } + + /** @return the compound Bloom filter block size from the configuration */ + public static int getBloomBlockSize(Configuration conf) { + return conf.getInt(IO_STOREFILE_BLOOM_BLOCK_SIZE, 128 * 1024); + } + + /** @return whether to cache compound Bloom filter chunks on write */ + public static boolean cacheChunksOnWrite(Configuration conf) { + return conf.getBoolean(IO_STOREFILE_BLOOM_CACHE_ON_WRITE, false); + } + +}; diff --git a/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java b/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java new file mode 100644 index 0000000..46691fb --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java @@ -0,0 +1,61 @@ +/* + * Copyright 2011 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.util; + +import org.apache.hadoop.io.Writable; + +/** + * Specifies methods needed to add elements to a Bloom filter and serialize the + * resulting Bloom filter as a sequence of bytes. + */ +public interface BloomFilterWriter extends BloomFilterBase { + + /** Allocate memory for the bloom filter data. */ + void allocBloom(); + + /** Compact the Bloom filter before writing metadata & data to disk. */ + void compactBloom(); + + /** + * Get a writable interface into bloom filter meta data. + * + * @return a writable instance that can be later written to a stream + */ + Writable getMetaWriter(); + + /** + * Get a writable interface into bloom filter data (the actual Bloom bits). + * Not used for compound Bloom filters. + * + * @return a writable instance that can be later written to a stream + */ + Writable getDataWriter(); + + /** + * Add the specified binary to the bloom filter. + * + * @param buf data to be added to the bloom + * @param offset offset into the data to be added + * @param len length of the data to be added + */ + void add(byte[] buf, int offset, int len); + +} diff --git a/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java b/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java index 7682834..6b6ae39 100644 --- a/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java +++ b/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java @@ -20,41 +20,71 @@ package org.apache.hadoop.hbase.util; +import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.Writable; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; +import java.text.NumberFormat; +import java.util.Random; /** * Implements a Bloom filter, as defined by Bloom in 1970. *

- * The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by - * the networking research community in the past decade thanks to the bandwidth efficiencies that it - * offers for the transmission of set membership information between networked hosts. A sender encodes - * the information into a bit vector, the Bloom filter, that is more compact than a conventional - * representation. Computation and space costs for construction are linear in the number of elements. - * The receiver uses the filter to test whether various elements are members of the set. Though the - * filter will occasionally return a false positive, it will never return a false negative. When creating - * the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size. + * The Bloom filter is a data structure that was introduced in 1970 and that has + * been adopted by the networking research community in the past decade thanks + * to the bandwidth efficiencies that it offers for the transmission of set + * membership information between networked hosts. A sender encodes the + * information into a bit vector, the Bloom filter, that is more compact than a + * conventional representation. Computation and space costs for construction are + * linear in the number of elements. The receiver uses the filter to test + * whether various elements are members of the set. Though the filter will + * occasionally return a false positive, it will never return a false negative. + * When creating the filter, the sender can choose its desired point in a + * trade-off between the false positive rate and the size. * *

- * Originally inspired by - * European Commission One-Lab Project 034819. + * Originally inspired by European Commission + * One-Lab Project 034819. + * + * Bloom filters are very sensitive to the number of elements inserted into + * them. For HBase, the number of entries depends on the size of the data stored + * in the column. Currently the default region size is 256MB, so entry count ~= + * 256MB / (average value size for column). Despite this rule of thumb, there is + * no efficient way to calculate the entry count after compactions. Therefore, + * it is often easier to use a dynamic bloom filter that will add extra space + * instead of allowing the error rate to grow. + * + * ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey + * .pdf ) + * + * m denotes the number of bits in the Bloom filter (bitSize) n denotes the + * number of elements inserted into the Bloom filter (maxKeys) k represents the + * number of hash functions used (nbHash) e represents the desired false + * positive rate for the bloom (err) + * + * If we fix the error rate (e) and know the number of entries, then the optimal + * bloom size m = -(n * ln(err) / (ln(2)^2) ~= n * ln(err) / ln(0.6185) + * + * The probability of false positives is minimized when k = m/n ln(2). * * @see BloomFilter The general behavior of a filter * - * @see Space/Time Trade-Offs in Hash Coding with Allowable Errors + * @see + * Space/Time Trade-Offs in Hash Coding with Allowable Errors */ -public class ByteBloomFilter implements BloomFilter { +public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { + /** Current file format version */ public static final int VERSION = 1; - /** Bytes (B) in the array */ + /** Bytes (B) in the array. This actually has to fit into an int. */ protected long byteSize; /** Number of hash functions */ - protected final int hashCount; + protected int hashCount; /** Hash type */ protected final int hashType; /** Hash Function */ @@ -66,6 +96,21 @@ public class ByteBloomFilter implements BloomFilter { /** Bloom bits */ protected ByteBuffer bloom; + /** Record separator for the Bloom filter statistics human-readable string */ + public static final String STATS_RECORD_SEP = "; "; + + /** + * Used in computing the optimal Bloom filter size. This approximately equals + * 0.480453. + */ + public static final double LOG2_SQUARED = Math.log(2) * Math.log(2); + + /** + * A random number generator to use for "fake lookups" when testing to + * estimate the ideal false positive rate. + */ + private static Random randomGeneratorForTest; + /** Bit-value lookup array to prevent doing the same work over and over */ private static final byte [] bitvals = { (byte) 0x01, @@ -76,86 +121,209 @@ public class ByteBloomFilter implements BloomFilter { (byte) 0x20, (byte) 0x40, (byte) 0x80 - }; + }; /** * Loads bloom filter meta data from file input. * @param meta stored bloom meta data * @throws IllegalArgumentException meta data is invalid */ - public ByteBloomFilter(ByteBuffer meta) - throws IllegalArgumentException { - int version = meta.getInt(); - if (version != VERSION) throw new IllegalArgumentException("Bad version"); - - this.byteSize = meta.getInt(); - this.hashCount = meta.getInt(); - this.hashType = meta.getInt(); - this.keyCount = meta.getInt(); + public ByteBloomFilter(DataInput meta) + throws IOException, IllegalArgumentException { + this.byteSize = meta.readInt(); + this.hashCount = meta.readInt(); + this.hashType = meta.readInt(); + this.keyCount = meta.readInt(); this.maxKeys = this.keyCount; this.hash = Hash.getInstance(this.hashType); + if (hash == null) { + throw new IllegalArgumentException("Invalid hash type: " + hashType); + } sanityCheck(); } /** - * Determines & initializes bloom filter meta data from user config. Call - * {@link #allocBloom()} to allocate bloom filter data. - * @param maxKeys Maximum expected number of keys that will be stored in this bloom - * @param errorRate Desired false positive error rate. Lower rate = more storage required - * @param hashType Type of hash function to use - * @param foldFactor When finished adding entries, you may be able to 'fold' - * this bloom to save space. Tradeoff potentially excess bytes in bloom for - * ability to fold if keyCount is exponentially greater than maxKeys. - * @throws IllegalArgumentException + * @param maxKeys + * @param errorRate + * @return the number of bits for a Bloom filter than can hold the given + * number of keys and provide the given error rate, assuming that the + * optimal number of hash functions is used and it does not have to + * be an integer. */ - public ByteBloomFilter(int maxKeys, float errorRate, int hashType, int foldFactor) - throws IllegalArgumentException { - /* - * Bloom filters are very sensitive to the number of elements inserted - * into them. For HBase, the number of entries depends on the size of the - * data stored in the column. Currently the default region size is 256MB, - * so entry count ~= 256MB / (average value size for column). Despite - * this rule of thumb, there is no efficient way to calculate the entry - * count after compactions. Therefore, it is often easier to use a - * dynamic bloom filter that will add extra space instead of allowing the - * error rate to grow. - * - * ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey.pdf ) - * - * m denotes the number of bits in the Bloom filter (bitSize) - * n denotes the number of elements inserted into the Bloom filter (maxKeys) - * k represents the number of hash functions used (nbHash) - * e represents the desired false positive rate for the bloom (err) - * - * If we fix the error rate (e) and know the number of entries, then - * the optimal bloom size m = -(n * ln(err) / (ln(2)^2) - * ~= n * ln(err) / ln(0.6185) - * - * The probability of false positives is minimized when k = m/n ln(2). - */ - long bitSize = (long)Math.ceil(maxKeys * (Math.log(errorRate) / Math.log(0.6185))); - int functionCount = (int)Math.ceil(Math.log(2) * (bitSize / maxKeys)); + public static long computeBitSize(long maxKeys, double errorRate) { + return (long) Math.ceil(maxKeys * (-Math.log(errorRate) / LOG2_SQUARED)); + } - // increase byteSize so folding is possible - long byteSize = (bitSize + 7) / 8; + /** + * The maximum number of keys we can put into a Bloom filter of a certain + * size to maintain the given error rate, assuming the number of hash + * functions is chosen optimally and does not even have to be an integer + * (hence the "ideal" in the function name). + * + * @param bitSize + * @param errorRate + * @return maximum number of keys that can be inserted into the Bloom filter + * @see {@link #computeMaxKeys(long, double, int)} for a more precise + * estimate + */ + public static long idealMaxKeys(long bitSize, double errorRate) { + // The reason we need to use floor here is that otherwise we might put + // more keys in a Bloom filter than is allowed by the target error rate. + return (long) (bitSize * (LOG2_SQUARED / -Math.log(errorRate))); + } + + /** + * The maximum number of keys we can put into a Bloom filter of a certain + * size to get the given error rate, with the given number of hash functions. + * + * @param bitSize + * @param errorRate + * @param hashCount + * @return the maximum number of keys that can be inserted in a Bloom filter + * to maintain the target error rate, if the number of hash functions + * is provided. + */ + public static long computeMaxKeys(long bitSize, double errorRate, + int hashCount) { + return (long) (-bitSize * 1.0 / hashCount * + Math.log(1 - Math.exp(Math.log(errorRate) / hashCount))); + } + + /** + * Computes the error rate for this Bloom filter, taking into account the + * actual number of hash functions and keys inserted. The return value of + * this function changes as a Bloom filter is being populated. Used for + * reporting the actual error rate of compound Bloom filters when writing + * them out. + * + * @return error rate for this particular Bloom filter + */ + public double actualErrorRate() { + return actualErrorRate(keyCount, byteSize * 8, hashCount); + } + + /** + * Computes the actual error rate for the given number of elements, number + * of bits, and number of hash functions. Taken directly from the + * Wikipedia Bloom filter article. + * + * @param maxKeys + * @param bitSize + * @param functionCount + * @return the actual error rate + */ + public static double actualErrorRate(long maxKeys, long bitSize, + int functionCount) { + return Math.exp(Math.log(1 - Math.exp(-functionCount * maxKeys * 1.0 + / bitSize)) * functionCount); + } + + /** + * Increases the given byte size of a Bloom filter until it can be folded by + * the given factor. + * + * @param bitSize + * @param foldFactor + * @return + */ + public static int computeFoldableByteSize(long bitSize, int foldFactor) { + long byteSizeLong = (bitSize + 7) / 8; int mask = (1 << foldFactor) - 1; - if ( (mask & byteSize) != 0) { - byteSize >>= foldFactor; - ++byteSize; - byteSize <<= foldFactor; + if ((mask & byteSizeLong) != 0) { + byteSizeLong >>= foldFactor; + ++byteSizeLong; + byteSizeLong <<= foldFactor; } + if (byteSizeLong > Integer.MAX_VALUE) { + throw new IllegalArgumentException("byteSize=" + byteSizeLong + " too " + + "large for bitSize=" + bitSize + ", foldFactor=" + foldFactor); + } + return (int) byteSizeLong; + } + + private static int optimalFunctionCount(int maxKeys, long bitSize) { + return (int) Math.ceil(Math.log(2) * (bitSize / maxKeys)); + } - this.byteSize = byteSize; - this.hashCount = functionCount; + /** Private constructor used by other constructors. */ + private ByteBloomFilter(int hashType) { this.hashType = hashType; - this.keyCount = 0; + this.hash = Hash.getInstance(hashType); + } + + /** + * Determines & initializes bloom filter meta data from user config. Call + * {@link #allocBloom()} to allocate bloom filter data. + * + * @param maxKeys Maximum expected number of keys that will be stored in this + * bloom + * @param errorRate Desired false positive error rate. Lower rate = more + * storage required + * @param hashType Type of hash function to use + * @param foldFactor When finished adding entries, you may be able to 'fold' + * this bloom to save space. Tradeoff potentially excess bytes in + * bloom for ability to fold if keyCount is exponentially greater + * than maxKeys. + * @throws IllegalArgumentException + */ + public ByteBloomFilter(int maxKeys, double errorRate, int hashType, + int foldFactor) throws IllegalArgumentException { + this(hashType); + + long bitSize = computeBitSize(maxKeys, errorRate); + hashCount = optimalFunctionCount(maxKeys, bitSize); this.maxKeys = maxKeys; - this.hash = Hash.getInstance(hashType); + // increase byteSize so folding is possible + byteSize = computeFoldableByteSize(bitSize, foldFactor); + sanityCheck(); } + /** + * Creates a Bloom filter of the given size. + * + * @param byteSizeHint the desired number of bytes for the Bloom filter bit + * array. Will be increased so that folding is possible. + * @param errorRate target false positive rate of the Bloom filter + * @param hashType Bloom filter hash function type + * @param foldFactor + * @return the new Bloom filter of the desired size + */ + public static ByteBloomFilter createBySize(int byteSizeHint, + double errorRate, int hashType, int foldFactor) { + ByteBloomFilter bbf = new ByteBloomFilter(hashType); + + bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8, foldFactor); + long bitSize = bbf.byteSize * 8; + bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate); + bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize); + + // Adjust max keys to bring error rate closer to what was requested, + // because byteSize was adjusted to allow for folding, and hashCount was + // rounded. + bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount); + + return bbf; + } + + /** + * Creates another similar Bloom filter. Does not copy the actual bits, and + * sets the new filter's key count to zero. + * + * @return a Bloom filter with the same configuration as this + */ + public ByteBloomFilter createAnother() { + ByteBloomFilter bbf = new ByteBloomFilter(hashType); + bbf.byteSize = byteSize; + bbf.hashCount = hashCount; + bbf.maxKeys = maxKeys; + return bbf; + } + @Override public void allocBloom() { if (this.bloom != null) { @@ -190,7 +358,6 @@ public class ByteBloomFilter implements BloomFilter { } } - @Override public void add(byte [] buf) { add(buf, 0, buf.length); } @@ -212,42 +379,64 @@ public class ByteBloomFilter implements BloomFilter { ++this.keyCount; } - /** - * Should only be used in tests when writing a bloom filter. - */ + /** Should only be used in tests */ boolean contains(byte [] buf) { return contains(buf, 0, buf.length, this.bloom); } - /** - * Should only be used in tests when writing a bloom filter. - */ + /** Should only be used in tests */ boolean contains(byte [] buf, int offset, int length) { - return contains(buf, offset, length, this.bloom); + return contains(buf, offset, length, bloom); } - @Override - public boolean contains(byte [] buf, ByteBuffer theBloom) { - return contains(buf, 0, buf.length, theBloom); + /** Should only be used in tests */ + boolean contains(byte[] buf, ByteBuffer bloom) { + return contains(buf, 0, buf.length, bloom); } @Override - public boolean contains(byte [] buf, int offset, int length, + public boolean contains(byte[] buf, int offset, int length, ByteBuffer theBloom) { + if (theBloom == null) { + // In a version 1 HFile Bloom filter data is stored in a separate meta + // block which is loaded on demand, but in version 2 it is pre-loaded. + // We want to use the same API in both cases. + theBloom = bloom; + } - if(theBloom.limit() != this.byteSize) { - throw new IllegalArgumentException("Bloom does not match expected size"); + if (theBloom.limit() != byteSize) { + throw new IllegalArgumentException("Bloom does not match expected size:" + + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize); } - int hash1 = this.hash.hash(buf, offset, length, 0); - int hash2 = this.hash.hash(buf, offset, length, hash1); + return contains(buf, offset, length, theBloom.array(), + theBloom.arrayOffset(), (int) byteSize, hash, hashCount); + } - for (int i = 0; i < this.hashCount; i++) { - long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8)); - if (!get(hashLoc, theBloom) ) { - return false; + public static boolean contains(byte[] buf, int offset, int length, + byte[] bloomArray, int bloomOffset, int bloomSize, Hash hash, + int hashCount) { + + int hash1 = hash.hash(buf, offset, length, 0); + int hash2 = hash.hash(buf, offset, length, hash1); + int bloomBitSize = bloomSize * 8; + + if (randomGeneratorForTest == null) { + // Production mode. + for (int i = 0; i < hashCount; i++) { + long hashLoc = Math.abs((hash1 + i * hash2) % bloomBitSize); + if (!get(hashLoc, bloomArray, bloomOffset)) + return false; + } + } else { + // Test mode with "fake lookups" to estimate "ideal false positive rate". + for (int i = 0; i < hashCount; i++) { + long hashLoc = randomGeneratorForTest.nextInt(bloomBitSize); + if (!get(hashLoc, bloomArray, bloomOffset)) + return false; } } + return true; } @@ -273,27 +462,31 @@ public class ByteBloomFilter implements BloomFilter { * @param pos index of bit * @return true if bit at specified index is 1, false if 0. */ - static boolean get(long pos, ByteBuffer theBloom) { + static boolean get(long pos, byte[] bloomArray, int bloomOffset) { int bytePos = (int)(pos / 8); int bitPos = (int)(pos % 8); - byte curByte = theBloom.get(bytePos); + byte curByte = bloomArray[bloomOffset + bytePos]; curByte &= bitvals[bitPos]; return (curByte != 0); } @Override - public int getKeyCount() { - return this.keyCount; + public long getKeyCount() { + return keyCount; } @Override - public int getMaxKeys() { - return this.maxKeys; + public long getMaxKeys() { + return maxKeys; } @Override - public int getByteSize() { - return (int)this.byteSize; + public long getByteSize() { + return byteSize; + } + + public int getHashType() { + return hashType; } @Override @@ -367,7 +560,7 @@ public class ByteBloomFilter implements BloomFilter { @Override public void write(DataOutput out) throws IOException { out.writeInt(VERSION); - out.writeInt((int)byteSize); + out.writeInt((int) byteSize); out.writeInt(hashCount); out.writeInt(hashType); out.writeInt(keyCount); @@ -387,4 +580,73 @@ public class ByteBloomFilter implements BloomFilter { } } + public int getHashCount() { + return hashCount; + } + + @Override + public boolean supportsAutoLoading() { + return bloom != null; + } + + public static void setFakeLookupMode(boolean enabled) { + if (enabled) { + randomGeneratorForTest = new Random(283742987L); + } else { + randomGeneratorForTest = null; + } + } + + /** + * {@inheritDoc} + * Just concatenate row and column by default. May return the original row + * buffer if the column qualifier is empty. + */ + @Override + public byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen, + byte[] qualBuf, int qualOffset, int qualLen) { + // Optimize the frequent case when only the row is provided. + if (qualLen <= 0 && rowOffset == 0 && rowLen == rowBuf.length) + return rowBuf; + + byte [] result = new byte[rowLen + qualLen]; + System.arraycopy(rowBuf, rowOffset, result, 0, rowLen); + if (qualLen > 0) + System.arraycopy(qualBuf, qualOffset, result, rowLen, qualLen); + return result; + } + + @Override + public RawComparator getComparator() { + return Bytes.BYTES_RAWCOMPARATOR; + } + + /** + * A human-readable string with statistics for the given Bloom filter. + * + * @param bloomFilter the Bloom filter to output statistics for; + * @return a string consisting of "<key>: <value>" parts + * separated by {@link #STATS_RECORD_SEP}. + */ + public static String formatStats(BloomFilterBase bloomFilter) { + StringBuilder sb = new StringBuilder(); + long k = bloomFilter.getKeyCount(); + long m = bloomFilter.getMaxKeys(); + + sb.append("BloomSize: " + bloomFilter.getByteSize() + STATS_RECORD_SEP); + sb.append("No of Keys in bloom: " + k + STATS_RECORD_SEP); + sb.append("Max Keys for bloom: " + m); + if (m > 0) { + sb.append(STATS_RECORD_SEP + "Percentage filled: " + + NumberFormat.getPercentInstance().format(k * 1.0 / m)); + } + return sb.toString(); + } + + @Override + public String toString() { + return formatStats(this) + STATS_RECORD_SEP + "Actual error rate: " + + String.format("%.8f", actualErrorRate()); + } + } diff --git a/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/src/main/java/org/apache/hadoop/hbase/util/Bytes.java index f5f742f..cc4b96d 100644 --- a/src/main/java/org/apache/hadoop/hbase/util/Bytes.java +++ b/src/main/java/org/apache/hadoop/hbase/util/Bytes.java @@ -308,18 +308,24 @@ public class Bytes { * @see #toStringBinary(byte[], int, int) */ public static String toStringBinary(final byte [] b) { + if (b == null) + return "null"; return toStringBinary(b, 0, b.length); } /** - * The same as {@link #toStringBinary(byte[])}, but returns a string "null" - * if given a null argument. + * Converts the given byte buffer, from its array offset to its limit, to + * a string. The position and the mark are ignored. + * + * @param buf a byte buffer + * @return a string representation of the buffer's binary contents */ - public static String toStringBinarySafe(final byte [] b) { - if (b == null) + public static String toStringBinary(ByteBuffer buf) { + if (buf == null) return "null"; - return toStringBinary(b, 0, b.length); + return toStringBinary(buf.array(), buf.arrayOffset(), buf.limit()); } + /** * Write a printable representation of a byte array. Non-printable * characters are hex escaped in the format \\x%02X, eg: @@ -1470,12 +1476,18 @@ public class Bytes { /** * Binary search for keys in indexes. + * * @param arr array of byte arrays to search for * @param key the key you want to find * @param offset the offset in the key you want to find * @param length the length of the key * @param comparator a comparator to compare. - * @return index of key + * @return zero-based index of the key, if the key is present in the array. + * Otherwise, a value -(i + 1) such that the key is between arr[i - + * 1] and arr[i] non-inclusively, where i is in [0, i], if we define + * arr[-1] = -Inf and arr[N] = Inf for an N-element array. The above + * means that this function can return 2N + 1 different values + * ranging from -(N + 1) to N - 1. */ public static int binarySearch(byte [][]arr, byte []key, int offset, int length, RawComparator comparator) { @@ -1584,4 +1596,34 @@ public class Bytes { return value; } + /** + * Writes a string as a fixed-size field, padded with zeros. + */ + public static void writeStringFixedSize(final DataOutput out, String s, + int size) throws IOException { + byte[] b = toBytes(s); + if (b.length > size) { + throw new IOException("Trying to write " + b.length + " bytes (" + + toStringBinary(b) + ") into a field of length " + size); + } + + out.writeBytes(s); + for (int i = 0; i < size - s.length(); ++i) + out.writeByte(0); + } + + /** + * Reads a fixed-size field and interprets it as a string padded with zeros. + */ + public static String readStringFixedSize(final DataInput in, int size) + throws IOException { + byte[] b = new byte[size]; + in.readFully(b); + int n = b.length; + while (n > 0 && b[n - 1] == 0) + --n; + + return toString(b, 0, n); + } + } diff --git a/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java b/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java new file mode 100644 index 0000000..c662763 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java @@ -0,0 +1,175 @@ +/* + * Copyright 2011 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.util; + +import java.io.DataInput; +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.io.hfile.FixedFileTrailer; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileBlock; +import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex; +import org.apache.hadoop.io.RawComparator; + +/** + * A Bloom filter implementation built on top of {@link ByteBloomFilter}, + * encapsulating a set of fixed-size Bloom filters written out at the time of + * {@link org.apache.hadoop.hbase.io.hfile.HFile} generation into the data + * block stream, and loaded on demand at query time. This class only provides + * reading capabilities. + */ +public class CompoundBloomFilter extends CompoundBloomFilterBase + implements BloomFilter { + + /** Used to load chunks on demand */ + private HFile.Reader reader; + + private HFileBlockIndex.BlockIndexReader index; + + private int hashCount; + private Hash hash; + + private long[] numQueriesPerChunk; + private long[] numPositivesPerChunk; + + /** + * De-serialization for compound Bloom filter metadata. Must be consistent + * with what {@link CompoundBloomFilterWriter} does. + * + * @param meta serialized Bloom filter metadata without any magic blocks + * @throws IOException + */ + public CompoundBloomFilter(DataInput meta, HFile.Reader reader) + throws IOException { + this.reader = reader; + + totalByteSize = meta.readLong(); + hashCount = meta.readInt(); + hashType = meta.readInt(); + totalKeyCount = meta.readLong(); + totalMaxKeys = meta.readLong(); + numChunks = meta.readInt(); + comparator = FixedFileTrailer.createComparator( + Bytes.toString(Bytes.readByteArray(meta))); + + hash = Hash.getInstance(hashType); + if (hash == null) { + throw new IllegalArgumentException("Invalid hash type: " + hashType); + } + + index = new HFileBlockIndex.BlockIndexReader(comparator, 1); + index.readRootIndex(meta, numChunks); + } + + @Override + public boolean contains(byte[] key, int keyOffset, int keyLength, + ByteBuffer bloom) { + // We try to store the result in this variable so we can update stats for + // testing, but when an error happens, we log a message and return. + boolean result; + + int block = index.rootBlockContainingKey(key, keyOffset, keyLength); + if (block < 0) { + result = false; // This key is not in the file. + } else { + HFileBlock bloomBlock; + try { + // We cache the block and use a positional read. + bloomBlock = reader.readBlock(index.getRootBlockOffset(block), + index.getRootBlockDataSize(block), true, true, false); + } catch (IOException ex) { + // The Bloom filter is broken, turn it off. + throw new IllegalArgumentException( + "Failed to load Bloom block for key " + + Bytes.toStringBinary(key, keyOffset, keyLength), ex); + } + + ByteBuffer bloomBuf = bloomBlock.getBufferReadOnly(); + result = ByteBloomFilter.contains(key, keyOffset, keyLength, + bloomBuf.array(), bloomBuf.arrayOffset() + HFileBlock.HEADER_SIZE, + bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount); + } + + if (numQueriesPerChunk != null && block >= 0) { + // Update statistics. Only used in unit tests. + ++numQueriesPerChunk[block]; + if (result) + ++numPositivesPerChunk[block]; + } + + return result; + } + + public boolean supportsAutoLoading() { + return true; + } + + public int getNumChunks() { + return numChunks; + } + + @Override + public RawComparator getComparator() { + return comparator; + } + + public void enableTestingStats() { + numQueriesPerChunk = new long[numChunks]; + numPositivesPerChunk = new long[numChunks]; + } + + public String formatTestingStats() { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < numChunks; ++i) { + sb.append("chunk #"); + sb.append(i); + sb.append(": queries="); + sb.append(numQueriesPerChunk[i]); + sb.append(", positives="); + sb.append(numPositivesPerChunk[i]); + sb.append(", positiveRatio="); + sb.append(numPositivesPerChunk[i] * 1.0 / numQueriesPerChunk[i]); + sb.append(";\n"); + } + return sb.toString(); + } + + public long getNumQueriesForTesting(int chunk) { + return numQueriesPerChunk[chunk]; + } + + public long getNumPositivesForTesting(int chunk) { + return numPositivesPerChunk[chunk]; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(ByteBloomFilter.formatStats(this)); + sb.append(ByteBloomFilter.STATS_RECORD_SEP + + "Number of chunks: " + numChunks); + sb.append(ByteBloomFilter.STATS_RECORD_SEP + + "Comparator: " + comparator.getClass().getSimpleName()); + return sb.toString(); + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java b/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java new file mode 100644 index 0000000..3f96e5a --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java @@ -0,0 +1,95 @@ +/* + * Copyright 2011 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.util; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.io.RawComparator; + +public class CompoundBloomFilterBase implements BloomFilterBase { + + /** + * At read time, the total number of chunks. At write time, the number of + * chunks created so far. The first chunk has an ID of 0, and the current + * chunk has the ID of numChunks - 1. + */ + protected int numChunks; + + /** + * The Bloom filter version. There used to be a DynamicByteBloomFilter which + * had version 2. + */ + public static final int VERSION = 3; + + /** Target error rate for configuring the filter and for information */ + protected float errorRate; + + /** The total number of keys in all chunks */ + protected long totalKeyCount; + protected long totalByteSize; + protected long totalMaxKeys; + + /** Hash function type to use, as defined in {@link Hash} */ + protected int hashType; + + /** Comparator used to compare Bloom filter keys */ + protected RawComparator comparator; + + @Override + public long getMaxKeys() { + return totalMaxKeys; + } + + @Override + public long getKeyCount() { + return totalKeyCount; + } + + @Override + public long getByteSize() { + return totalByteSize; + } + + private static final byte[] DUMMY = new byte[0]; + + /** + * Prepare an ordered pair of row and qualifier to be compared using + * {@link KeyValue.KeyComparator}. This is only used for row-column Bloom + * filters. + */ + @Override + public byte[] createBloomKey(byte[] row, int roffset, int rlength, + byte[] qualifier, int qoffset, int qlength) { + if (qualifier == null) + qualifier = DUMMY; + + // Make sure this does not specify a timestamp so that the default maximum + // (most recent) timestamp is used. + KeyValue kv = KeyValue.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0, + qualifier, qoffset, qlength); + return kv.getKey(); + } + + @Override + public RawComparator getComparator() { + return comparator; + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java b/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java new file mode 100644 index 0000000..8571d9c --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java @@ -0,0 +1,277 @@ +/* + * Copyright 2011 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.util; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.Queue; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.io.hfile.BlockType; +import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex; +import org.apache.hadoop.hbase.io.hfile.InlineBlockWriter; +import org.apache.hadoop.io.RawComparator; +import org.apache.hadoop.io.Writable; + +/** + * Adds methods required for writing a compound Bloom filter to the data + * section of an {@link org.apache.hadoop.hbase.io.hfile.HFile} to the + * {@link CompoundBloomFilter} class. + */ +public class CompoundBloomFilterWriter extends CompoundBloomFilterBase + implements BloomFilterWriter, InlineBlockWriter { + + protected static final Log LOG = + LogFactory.getLog(CompoundBloomFilterWriter.class); + + /** The current chunk being written to */ + private ByteBloomFilter chunk; + + /** Previous chunk, so that we can create another similar chunk */ + private ByteBloomFilter prevChunk; + + /** Maximum fold factor */ + private int maxFold; + + /** The size of individual Bloom filter chunks to create */ + private int chunkByteSize; + + /** A Bloom filter chunk enqueued for writing */ + private static class ReadyChunk { + int chunkId; + byte[] firstKey; + ByteBloomFilter chunk; + } + + private Queue readyChunks = new LinkedList(); + + /** The first key in the current Bloom filter chunk. */ + private byte[] firstKeyInChunk = null; + + private HFileBlockIndex.BlockIndexWriter bloomBlockIndexWriter = + new HFileBlockIndex.BlockIndexWriter(); + + /** Whether to cache-on-write compound Bloom filter chunks */ + private boolean cacheOnWrite; + + /** + * @param chunkByteSizeHint + * each chunk's size in bytes. The real chunk size might be different + * as required by the fold factor. + * @param errorRate + * target false positive rate + * @param hashType + * hash function type to use + * @param maxFold + * maximum degree of folding allowed + */ + public CompoundBloomFilterWriter(int chunkByteSizeHint, float errorRate, + int hashType, int maxFold, boolean cacheOnWrite, + RawComparator comparator) { + chunkByteSize = ByteBloomFilter.computeFoldableByteSize( + chunkByteSizeHint * 8, maxFold); + + this.errorRate = errorRate; + this.hashType = hashType; + this.maxFold = maxFold; + this.cacheOnWrite = cacheOnWrite; + this.comparator = comparator; + } + + @Override + public boolean shouldWriteBlock(boolean closing) { + enqueueReadyChunk(closing); + return !readyChunks.isEmpty(); + } + + /** + * Enqueue the current chunk if it is ready to be written out. + * + * @param closing true if we are closing the file, so we do not expect new + * keys to show up + */ + private void enqueueReadyChunk(boolean closing) { + if (chunk == null || + (chunk.getKeyCount() < chunk.getMaxKeys() && !closing)) { + return; + } + + if (firstKeyInChunk == null) { + throw new NullPointerException("Trying to enqueue a chunk, " + + "but first key is null: closing=" + closing + ", keyCount=" + + chunk.getKeyCount() + ", maxKeys=" + chunk.getMaxKeys()); + } + + ReadyChunk readyChunk = new ReadyChunk(); + readyChunk.chunkId = numChunks - 1; + readyChunk.chunk = chunk; + readyChunk.firstKey = firstKeyInChunk; + readyChunks.add(readyChunk); + + long prevMaxKeys = chunk.getMaxKeys(); + long prevByteSize = chunk.getByteSize(); + + chunk.compactBloom(); + + if (LOG.isDebugEnabled() && prevByteSize != chunk.getByteSize()) { + LOG.debug("Compacted Bloom chunk #" + readyChunk.chunkId + " from [" + + prevMaxKeys + " max keys, " + prevByteSize + " bytes] to [" + + chunk.getMaxKeys() + " max keys, " + chunk.getByteSize() + + " bytes]"); + } + + totalMaxKeys += chunk.getMaxKeys(); + totalByteSize += chunk.getByteSize(); + + firstKeyInChunk = null; + prevChunk = chunk; + chunk = null; + } + + /** + * Adds a Bloom filter key. This key must be greater than the previous key, + * as defined by the comparator this compound Bloom filter is configured + * with. For efficiency, key monotonicity is not checked here. See + * {@link org.apache.hadoop.hbase.regionserver.StoreFile.Writer#append( + * org.apache.hadoop.hbase.KeyValue)} for the details of deduplication. + */ + @Override + public void add(byte[] bloomKey, int keyOffset, int keyLength) { + if (bloomKey == null) + throw new NullPointerException(); + + enqueueReadyChunk(false); + + if (chunk == null) { + if (firstKeyInChunk != null) { + throw new IllegalStateException("First key in chunk already set: " + + Bytes.toStringBinary(firstKeyInChunk)); + } + firstKeyInChunk = Arrays.copyOfRange(bloomKey, keyOffset, keyOffset + + keyLength); + + if (prevChunk == null) { + // First chunk + chunk = ByteBloomFilter.createBySize(chunkByteSize, errorRate, + hashType, maxFold); + } else { + // Use the same parameters as the last chunk, but a new array and + // a zero key count. + chunk = prevChunk.createAnother(); + } + + if (chunk.getKeyCount() != 0) { + throw new IllegalStateException("keyCount=" + chunk.getKeyCount() + + " > 0"); + } + + chunk.allocBloom(); + ++numChunks; + } + + chunk.add(bloomKey, keyOffset, keyLength); + ++totalKeyCount; + } + + @Override + public void writeInlineBlock(DataOutput out) throws IOException { + // We don't remove the chunk from the queue here, because we might need it + // again for cache-on-write. + ReadyChunk readyChunk = readyChunks.peek(); + + ByteBloomFilter readyChunkBloom = readyChunk.chunk; + readyChunkBloom.getDataWriter().write(out); + } + + @Override + public void blockWritten(long offset, int onDiskSize, int uncompressedSize) { + ReadyChunk readyChunk = readyChunks.remove(); + bloomBlockIndexWriter.addEntry(readyChunk.firstKey, offset, onDiskSize); + } + + @Override + public BlockType getInlineBlockType() { + return BlockType.BLOOM_CHUNK; + } + + private class MetaWriter implements Writable { + protected MetaWriter() {} + + @Override + public void readFields(DataInput in) throws IOException { + throw new IOException("Cant read with this class."); + } + + /** + * This is modeled after {@link ByteBloomFilter.MetaWriter} for simplicity, + * although the two metadata formats do not have to be consistent. This + * does have to be consistent with how {@link + * CompoundBloomFilter#CompoundBloomFilter(DataInput, + * org.apache.hadoop.hbase.io.hfile.HFile.Reader)} reads fields. + */ + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(VERSION); + + out.writeLong(getByteSize()); + out.writeInt(prevChunk.getHashCount()); + out.writeInt(prevChunk.getHashType()); + out.writeLong(getKeyCount()); + out.writeLong(getMaxKeys()); + + // Fields that don't have equivalents in ByteBloomFilter. + out.writeInt(numChunks); + Bytes.writeByteArray(out, + Bytes.toBytes(comparator.getClass().getName())); + + // Write a single-level index without compression or block header. + bloomBlockIndexWriter.writeSingleLevelIndex(out, "Bloom filter"); + } + } + + @Override + public Writable getMetaWriter() { + return new MetaWriter(); + } + + @Override + public void compactBloom() { + } + + @Override + public void allocBloom() { + // Nothing happens here. All allocation happens on demand. + } + + @Override + public Writable getDataWriter() { + return null; + } + + @Override + public boolean cacheOnWrite() { + return cacheOnWrite; + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java b/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java index 2d09056..aca820f 100644 --- a/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java +++ b/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java @@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hdfs.DistributedFileSystem; @@ -80,6 +81,7 @@ public class CompressionTest { } } + Configuration conf = HBaseConfiguration.create(); try { Compressor c = algo.getCompressor(); algo.returnCompressor(c); @@ -103,13 +105,14 @@ public class CompressionTest { public static void doSmokeTest(FileSystem fs, Path path, String codec) throws Exception { - HFile.Writer writer = new HFile.Writer( + Configuration conf = HBaseConfiguration.create(); + HFile.Writer writer = HFile.getWriterFactory(conf).createWriter( fs, path, HFile.DEFAULT_BLOCKSIZE, codec, null); writer.append(Bytes.toBytes("testkey"), Bytes.toBytes("testval")); writer.appendFileInfo(Bytes.toBytes("infokey"), Bytes.toBytes("infoval")); writer.close(); - HFile.Reader reader = new HFile.Reader(fs, path, null, false, false); + HFile.Reader reader = HFile.createReader(fs, path, null, false, false); reader.loadFileInfo(); byte[] key = reader.getFirstKey(); boolean rc = Bytes.toString(key).equals("testkey"); diff --git a/src/main/java/org/apache/hadoop/hbase/util/DynamicByteBloomFilter.java b/src/main/java/org/apache/hadoop/hbase/util/DynamicByteBloomFilter.java deleted file mode 100644 index 441167b..0000000 --- a/src/main/java/org/apache/hadoop/hbase/util/DynamicByteBloomFilter.java +++ /dev/null @@ -1,302 +0,0 @@ -/* - * Copyright 2010 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.util; - -import org.apache.hadoop.io.Writable; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.nio.ByteBuffer; - -/** - * Implements a dynamic Bloom filter, as defined in the INFOCOM 2006 paper. - *

- * A dynamic Bloom filter (DBF) makes use of a s * m bit matrix but - * each of the s rows is a standard Bloom filter. The creation - * process of a DBF is iterative. At the start, the DBF is a 1 * m - * bit matrix, i.e., it is composed of a single standard Bloom filter. - * It assumes that nr elements are recorded in the - * initial bit vector, where nr <= n (n is - * the cardinality of the set A to record in the filter). - *

- * As the size of A grows during the execution of the application, - * several keys must be inserted in the DBF. When inserting a key into the DBF, - * one must first get an active Bloom filter in the matrix. A Bloom filter is - * active when the number of recorded keys, nr, is - * strictly less than the current cardinality of A, n. - * If an active Bloom filter is found, the key is inserted and - * nr is incremented by one. On the other hand, if there - * is no active Bloom filter, a new one is created (i.e., a new row is added to - * the matrix) according to the current size of A and the element - * is added in this new Bloom filter and the nr value of - * this new Bloom filter is set to one. A given key is said to belong to the - * DBF if the k positions are set to one in one of the matrix rows. - *

- * Originally created by - * European Commission One-Lab Project 034819. - * - * @see BloomFilter A Bloom filter - * - * @see Theory and Network Applications of Dynamic Bloom Filters - */ -public class DynamicByteBloomFilter implements BloomFilter { - /** Current file format version */ - public static final int VERSION = 2; - /** Maximum number of keys in a dynamic Bloom filter row. */ - protected final int keyInterval; - /** The maximum false positive rate per bloom */ - protected final float errorRate; - /** Hash type */ - protected final int hashType; - /** The number of keys recorded in the current Bloom filter. */ - protected int curKeys; - /** expected size of bloom filter matrix (used during reads) */ - protected int readMatrixSize; - /** The matrix of Bloom filters (contains bloom data only during writes). */ - protected ByteBloomFilter[] matrix; - - /** - * Normal read constructor. Loads bloom filter meta data. - * @param meta stored bloom meta data - * @throws IllegalArgumentException meta data is invalid - */ - public DynamicByteBloomFilter(ByteBuffer meta) throws IllegalArgumentException { - int version = meta.getInt(); - if (version != VERSION) throw new IllegalArgumentException("Bad version"); - - this.keyInterval = meta.getInt(); - this.errorRate = meta.getFloat(); - this.hashType = meta.getInt(); - this.readMatrixSize = meta.getInt(); - this.curKeys = meta.getInt(); - - readSanityCheck(); - - this.matrix = new ByteBloomFilter[1]; - this.matrix[0] = new ByteBloomFilter(keyInterval, errorRate, hashType, 0); -} - - /** - * Normal write constructor. Note that this doesn't allocate bloom data by - * default. Instead, call allocBloom() before adding entries. - * @param errorRate - * @param hashType type of the hashing function (see org.apache.hadoop.util.hash.Hash). - * @param keyInterval Maximum number of keys to record per Bloom filter row. - * @throws IllegalArgumentException The input parameters were invalid - */ - public DynamicByteBloomFilter(int keyInterval, float errorRate, int hashType) - throws IllegalArgumentException { - this.keyInterval = keyInterval; - this.errorRate = errorRate; - this.hashType = hashType; - this.curKeys = 0; - - if(keyInterval <= 0) { - throw new IllegalArgumentException("keyCount must be > 0"); - } - - this.matrix = new ByteBloomFilter[1]; - this.matrix[0] = new ByteBloomFilter(keyInterval, errorRate, hashType, 0); -} - - @Override - public void allocBloom() { - this.matrix[0].allocBloom(); - } - - void readSanityCheck() throws IllegalArgumentException { - if (this.curKeys <= 0) { - throw new IllegalArgumentException("last bloom's key count invalid"); - } - - if (this.readMatrixSize <= 0) { - throw new IllegalArgumentException("matrix size must be known"); - } - } - - @Override - public void add(byte []buf, int offset, int len) { - BloomFilter bf = getCurBloom(); - - if (bf == null) { - addRow(); - bf = matrix[matrix.length - 1]; - curKeys = 0; - } - - bf.add(buf, offset, len); - curKeys++; - } - - @Override - public void add(byte []buf) { - add(buf, 0, buf.length); - } - - /** - * Should only be used in tests when writing a bloom filter. - */ - boolean contains(byte [] buf) { - return contains(buf, 0, buf.length); - } - - /** - * Should only be used in tests when writing a bloom filter. - */ - boolean contains(byte [] buf, int offset, int length) { - for (int i = 0; i < matrix.length; i++) { - if (matrix[i].contains(buf, offset, length)) { - return true; - } - } - return false; - } - - @Override - public boolean contains(byte [] buf, ByteBuffer theBloom) { - return contains(buf, 0, buf.length, theBloom); - } - - @Override - public boolean contains(byte[] buf, int offset, int length, - ByteBuffer theBloom) { - if(offset + length > buf.length) { - return false; - } - - // current version assumes uniform size - int bytesPerBloom = this.matrix[0].getByteSize(); - - if(theBloom.limit() != bytesPerBloom * readMatrixSize) { - throw new IllegalArgumentException("Bloom does not match expected size"); - } - - ByteBuffer tmp = theBloom.duplicate(); - - // note: actually searching an array of blooms that have been serialized - for (int m = 0; m < readMatrixSize; ++m) { - tmp.position(m* bytesPerBloom); - tmp.limit(tmp.position() + bytesPerBloom); - boolean match = this.matrix[0].contains(buf, offset, length, tmp.slice()); - if (match) { - return true; - } - } - - // matched no bloom filters - return false; - } - - int bloomCount() { - return Math.max(this.matrix.length, this.readMatrixSize); - } - - @Override - public int getKeyCount() { - return (bloomCount()-1) * this.keyInterval + this.curKeys; - } - - @Override - public int getMaxKeys() { - return bloomCount() * this.keyInterval; - } - - @Override - public int getByteSize() { - return bloomCount() * this.matrix[0].getByteSize(); - } - - @Override - public void compactBloom() { - } - - /** - * Adds a new row to this dynamic Bloom filter. - */ - private void addRow() { - ByteBloomFilter[] tmp = new ByteBloomFilter[matrix.length + 1]; - - for (int i = 0; i < matrix.length; i++) { - tmp[i] = matrix[i]; - } - - tmp[tmp.length-1] = new ByteBloomFilter(keyInterval, errorRate, hashType, 0); - tmp[tmp.length-1].allocBloom(); - matrix = tmp; - } - - /** - * Returns the currently-unfilled row in the dynamic Bloom Filter array. - * @return BloomFilter The active standard Bloom filter. - * Null otherwise. - */ - private BloomFilter getCurBloom() { - if (curKeys >= keyInterval) { - return null; - } - - return matrix[matrix.length - 1]; - } - - @Override - public Writable getMetaWriter() { - return new MetaWriter(); - } - - @Override - public Writable getDataWriter() { - return new DataWriter(); - } - - private class MetaWriter implements Writable { - protected MetaWriter() {} - @Override - public void readFields(DataInput arg0) throws IOException { - throw new IOException("Cant read with this class."); - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeInt(VERSION); - out.writeInt(keyInterval); - out.writeFloat(errorRate); - out.writeInt(hashType); - out.writeInt(matrix.length); - out.writeInt(curKeys); - } - } - - private class DataWriter implements Writable { - protected DataWriter() {} - @Override - public void readFields(DataInput arg0) throws IOException { - throw new IOException("Cant read with this class."); - } - - @Override - public void write(DataOutput out) throws IOException { - for (int i = 0; i < matrix.length; ++i) { - matrix[i].writeBloom(out); - } - } - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/util/Hash.java b/src/main/java/org/apache/hadoop/hbase/util/Hash.java index 0a533d9..8a3dcf5 100644 --- a/src/main/java/org/apache/hadoop/hbase/util/Hash.java +++ b/src/main/java/org/apache/hadoop/hbase/util/Hash.java @@ -122,8 +122,8 @@ public abstract class Hash { } /** - * Calculate a hash using bytes from 0 to length, and - * the provided seed value + * Calculate a hash using bytes from offset to offset + + * length, and the provided seed value. * @param bytes input bytes * @param offset the offset into the array to start consideration * @param length length of the valid bytes after offset to consider diff --git a/src/main/java/org/apache/hadoop/hbase/util/IdLock.java b/src/main/java/org/apache/hadoop/hbase/util/IdLock.java new file mode 100644 index 0000000..e9202dd --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/util/IdLock.java @@ -0,0 +1,120 @@ +/* + * Copyright 2011 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.util; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** + * Allows multiple concurrent clients to lock on a numeric id with a minimal + * memory overhead. The intended usage is as follows: + * + *

+ * IdLock.Entry lockEntry = idLock.getLockEntry(id);
+ * try {
+ *   // User code.
+ * } finally {
+ *   idLock.releaseLockEntry(lockEntry);
+ * }
+ */ +public class IdLock { + + /** An entry returned to the client as a lock object */ + public static class Entry { + private final long id; + private int numWaiters; + private boolean isLocked = true; + + private Entry(long id) { + this.id = id; + } + + public String toString() { + return "id=" + id + ", numWaiter=" + numWaiters + ", isLocked=" + + isLocked; + } + } + + private ConcurrentMap map = + new ConcurrentHashMap(); + + /** + * Blocks until the lock corresponding to the given id is acquired. + * + * @param id an arbitrary number to lock on + * @return an "entry" to pass to {@link #releaseLockEntry(Entry)} to release + * the lock + * @throws IOException if interrupted + */ + public Entry getLockEntry(long id) throws IOException { + Entry entry = new Entry(id); + Entry existing; + while ((existing = map.putIfAbsent(entry.id, entry)) != null) { + synchronized (existing) { + if (existing.isLocked) { + ++existing.numWaiters; // Add ourselves to waiters. + while (existing.isLocked) { + try { + existing.wait(); + } catch (InterruptedException e) { + --existing.numWaiters; // Remove ourselves from waiters. + throw new InterruptedIOException( + "Interrupted waiting to acquire sparse lock"); + } + } + + --existing.numWaiters; // Remove ourselves from waiters. + existing.isLocked = true; + return existing; + } + // If the entry is not locked, it might already be deleted from the + // map, so we cannot return it. We need to get our entry into the map + // or get someone else's locked entry. + } + } + return entry; + } + + /** + * Must be called in a finally block to decrease the internal counter and + * remove the monitor object for the given id if the caller is the last + * client. + * + * @param entry the return value of {@link #getLockEntry(long)} + */ + public void releaseLockEntry(Entry entry) { + synchronized (entry) { + entry.isLocked = false; + if (entry.numWaiters > 0) { + entry.notify(); + } else { + map.remove(entry.id); + } + } + } + + /** For testing */ + void assertMapEmpty() { + assert map.size() == 0; + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java index d8cc8aa..d456823 100644 --- a/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java +++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java @@ -487,8 +487,8 @@ public class RecoverableZooKeeper { return path; } LOG.error("Node " + path + " already exists with " + - Bytes.toStringBinarySafe(currentData) + ", could not write " + - Bytes.toStringBinarySafe(data)); + Bytes.toStringBinary(currentData) + ", could not write " + + Bytes.toStringBinary(data)); throw e; } LOG.error("Node " + path + " already exists and this is not a " + diff --git a/src/main/resources/hbase-default.xml b/src/main/resources/hbase-default.xml index 6e5d190..66548ca 100644 --- a/src/main/resources/hbase-default.xml +++ b/src/main/resources/hbase-default.xml @@ -447,6 +447,57 @@ + hfile.block.index.cacheonwrite + false + + This allows to put non-root multi-level index blocks into the block + cache at the time the index is being written. + + + + hfile.index.block.max.size + 131072 + + When the size of a leaf-level, intermediate-level, or root-level + index block in a multi-level block index grows to this size, the + block is written out and a new block is started. + + + + hfile.format.version + 2 + + The HFile format version to use for new files. Set this to 1 to test + backwards-compatibility. The default value of this option should be + consistent with FixedFileTrailer.MAX_VERSION. + + + + io.storefile.bloom.block.size + 131072 + + The size in bytes of a single block ("chunk") of a compound Bloom + filter. This size is approximate, because Bloom blocks can only be + inserted at data block boundaries, and the number of keys per data + block varies. + + + + io.storefile.bloom.cacheonwrite + false + + Enables cache-on-write for inline blocks of a compound Bloom filter. + + + + hbase.rs.cacheblocksonwrite + false + + Whether an HFile block should be added to the block cache when the + block is finished. + + + hbase.rpc.engine org.apache.hadoop.hbase.ipc.WritableRpcEngine Implementation of org.apache.hadoop.hbase.ipc.RpcEngine to be diff --git a/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 57a100b..88d6ad7 100644 --- a/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.migration.HRegionInfo090x; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -108,6 +109,19 @@ public class HBaseTestingUtility { */ public static final String DEFAULT_TEST_DIRECTORY = "target/test-data"; + /** Compression algorithms to use in parameterized JUnit 4 tests */ + public static final List COMPRESSION_ALGORITHMS_PARAMETERIZED = + Arrays.asList(new Object[][] { + { Compression.Algorithm.NONE }, + { Compression.Algorithm.GZ } + }); + + /** Compression algorithms to use in testing */ + public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = + new Compression.Algorithm[] { + Compression.Algorithm.NONE, Compression.Algorithm.GZ + }; + public HBaseTestingUtility() { this(HBaseConfiguration.create()); } @@ -136,6 +150,18 @@ public class HBaseTestingUtility { } /** + * Makes sure the test directory is set up so that {@link #getTestDir()} + * returns a valid directory. Useful in unit tests that do not run a + * mini-cluster. + */ + public void initTestDir() { + if (System.getProperty(TEST_DIRECTORY_KEY) == null) { + clusterTestBuildDir = setupClusterTestBuildDir(); + System.setProperty(TEST_DIRECTORY_KEY, clusterTestBuildDir.getPath()); + } + } + + /** * @return Where to write test data on local filesystem; usually * {@link #DEFAULT_TEST_DIRECTORY} * @see #setupClusterTestBuildDir() diff --git a/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java b/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java index ad29e60..e29b8af 100644 --- a/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java +++ b/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java @@ -188,8 +188,9 @@ public class HFilePerformanceEvaluation { @Override void setUp() throws Exception { - writer = new HFile.Writer(this.fs, this.mf, RFILE_BLOCKSIZE, - (Compression.Algorithm) null, null, null); + writer = HFile.getWriterFactory(conf).createWriter(this.fs, this.mf, + RFILE_BLOCKSIZE, + (Compression.Algorithm) null, null); } @Override @@ -225,7 +226,7 @@ public class HFilePerformanceEvaluation { @Override void setUp() throws Exception { - reader = new HFile.Reader(this.fs, this.mf, null, false, false); + reader = HFile.createReader(this.fs, this.mf, null, false, false); this.reader.loadFileInfo(); } diff --git a/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java b/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java index 7cb4d9d..94289c0 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java +++ b/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.io; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -60,9 +61,11 @@ public class TestHalfStoreFileReader { String root_dir = HBaseTestingUtility.getTestDir("TestHalfStoreFile").toString(); Path p = new Path(root_dir, "test"); - FileSystem fs = FileSystem.get(test_util.getConfiguration()); + Configuration conf = test_util.getConfiguration(); + FileSystem fs = FileSystem.get(conf); - HFile.Writer w = new HFile.Writer(fs, p, 1024, "none", KeyValue.KEY_COMPARATOR); + HFile.Writer w = HFile.getWriterFactory(conf).createWriter(fs, p, 1024, + "none", KeyValue.KEY_COMPARATOR); // write some things. List items = genSomeKeys(); @@ -71,7 +74,7 @@ public class TestHalfStoreFileReader { } w.close(); - HFile.Reader r = new HFile.Reader(fs, p, null, false, false); + HFile.Reader r = HFile.createReader(fs, p, null, false, false); r.loadFileInfo(); byte [] midkey = r.midkey(); KeyValue midKV = KeyValue.createKeyValueFromKey(midkey); diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java index 7e9fbfd..2d83a76 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java @@ -67,9 +67,9 @@ public class RandomSeek { Path path = new Path("/Users/ryan/rfile.big.txt"); long start = System.currentTimeMillis(); SimpleBlockCache cache = new SimpleBlockCache(); - Reader reader = new HFile.Reader(lfs, path, cache, false, false); + Reader reader = HFile.createReader(lfs, path, cache, false, false); reader.loadFileInfo(); - System.out.println(reader.trailer); + System.out.println(reader.getTrailer()); long end = System.currentTimeMillis(); System.out.println("Index read time: " + (end - start)); diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java new file mode 100644 index 0000000..5e98375 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java @@ -0,0 +1,229 @@ +/* + * Copyright 2011 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.util.ArrayList; +import java.util.Collection; +import java.util.EnumMap; +import java.util.List; +import java.util.Random; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.BlockCache; +import org.apache.hadoop.hbase.io.hfile.Compression; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileBlock; +import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex; +import org.apache.hadoop.hbase.io.hfile.HFileReaderV2; +import org.apache.hadoop.hbase.io.hfile.HFileScanner; +import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.util.BloomFilterFactory; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; +import static org.junit.Assert.*; + +/** + * Tests {@link HFile} cache-on-write functionality for the following block + * types: data blocks, non-root index blocks, and Bloom filter blocks. + */ +@RunWith(Parameterized.class) +public class TestCacheOnWrite { + + private static final Log LOG = LogFactory.getLog(TestCacheOnWrite.class); + + private static final HBaseTestingUtility TEST_UTIL = + new HBaseTestingUtility(); + private Configuration conf; + private FileSystem fs; + private Random rand = new Random(12983177L); + private Path storeFilePath; + private Compression.Algorithm compress; + private CacheOnWriteType cowType; + private BlockCache blockCache; + private String testName; + + private static final int DATA_BLOCK_SIZE = 2048; + private static final int NUM_KV = 25000; + private static final int INDEX_BLOCK_SIZE = 512; + private static final int BLOOM_BLOCK_SIZE = 4096; + + /** The number of valid key types possible in a store file */ + private static final int NUM_VALID_KEY_TYPES = + KeyValue.Type.values().length - 2; + + private static enum CacheOnWriteType { + DATA_BLOCKS(BlockType.DATA, HFile.CACHE_BLOCKS_ON_WRITE_KEY), + BLOOM_BLOCKS(BlockType.BLOOM_CHUNK, + BloomFilterFactory.IO_STOREFILE_BLOOM_CACHE_ON_WRITE), + INDEX_BLOCKS(BlockType.LEAF_INDEX, + HFileBlockIndex.CACHE_INDEX_BLOCKS_ON_WRITE_KEY); + + private final String confKey; + private final BlockType inlineBlockType; + + private CacheOnWriteType(BlockType inlineBlockType, String confKey) { + this.inlineBlockType = inlineBlockType; + this.confKey = confKey; + } + + public boolean shouldBeCached(BlockType blockType) { + return blockType == inlineBlockType + || blockType == BlockType.INTERMEDIATE_INDEX + && inlineBlockType == BlockType.LEAF_INDEX; + } + + public void modifyConf(Configuration conf) { + for (CacheOnWriteType cowType : CacheOnWriteType.values()) + conf.setBoolean(cowType.confKey, cowType == this); + } + + } + + public TestCacheOnWrite(CacheOnWriteType cowType, + Compression.Algorithm compress) { + this.cowType = cowType; + this.compress = compress; + testName = "[cacheOnWrite=" + cowType + ", compress=" + compress + "]"; + } + + @Parameters + public static Collection getParameters() { + List cowTypes = new ArrayList(); + for (CacheOnWriteType cowType : CacheOnWriteType.values()) + for (Compression.Algorithm compress : + HBaseTestingUtility.COMPRESSION_ALGORITHMS) { + cowTypes.add(new Object[] { cowType, compress }); + } + return cowTypes; + } + + @Before + public void setUp() throws IOException { + conf = TEST_UTIL.getConfiguration(); + conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION); + conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, INDEX_BLOCK_SIZE); + conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, + BLOOM_BLOCK_SIZE); + cowType.modifyConf(conf); + fs = FileSystem.get(conf); + blockCache = StoreFile.getBlockCache(conf); + } + + @After + public void tearDown() { + blockCache.evictBlocksByPrefix(""); + } + + @Test + public void testCacheOnWrite() throws IOException { + writeStoreFile(); + readStoreFile(); + } + + private void readStoreFile() throws IOException { + HFileReaderV2 reader = (HFileReaderV2) HFile.createReader(fs, + storeFilePath, null, false, false); + LOG.info("HFile information: " + reader); + HFileScanner scanner = reader.getScanner(false, false); + assertTrue(testName, scanner.seekTo()); + + long offset = 0; + HFileBlock prevBlock = null; + EnumMap blockCountByType = + new EnumMap(BlockType.class); + + while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) { + HFileBlock block = reader.readBlockData(offset, prevBlock == null ? -1 + : prevBlock.getNextBlockOnDiskSizeWithHeader(), -1, false); + String blockCacheKey = HFile.getBlockCacheKey(reader.getName(), offset); + boolean isCached = blockCache.getBlock(blockCacheKey, true) != null; + boolean shouldBeCached = cowType.shouldBeCached(block.getBlockType()); + assertEquals(testName + " " + block, shouldBeCached, isCached); + prevBlock = block; + offset += block.getOnDiskSizeWithHeader(); + BlockType bt = block.getBlockType(); + Integer count = blockCountByType.get(bt); + blockCountByType.put(bt, (count == null ? 0 : count) + 1); + } + + LOG.info("Block count by type: " + blockCountByType); + assertEquals( + "{DATA=1367, LEAF_INDEX=172, BLOOM_CHUNK=9, INTERMEDIATE_INDEX=24}", + blockCountByType.toString()); + + reader.close(); + } + + public static KeyValue.Type generateKeyType(Random rand) { + if (rand.nextBoolean()) { + // Let's make half of KVs puts. + return KeyValue.Type.Put; + } else { + KeyValue.Type keyType = + KeyValue.Type.values()[1 + rand.nextInt(NUM_VALID_KEY_TYPES)]; + if (keyType == KeyValue.Type.Minimum || keyType == KeyValue.Type.Maximum) + { + throw new RuntimeException("Generated an invalid key type: " + keyType + + ". " + "Probably the layout of KeyValue.Type has changed."); + } + return keyType; + } + } + + public void writeStoreFile() throws IOException { + Path storeFileParentDir = new Path(HBaseTestingUtility.getTestDir(), + "test_cache_on_write"); + StoreFile.Writer sfw = StoreFile.createWriter(fs, storeFileParentDir, + DATA_BLOCK_SIZE, compress, KeyValue.COMPARATOR, conf, + StoreFile.BloomType.ROWCOL, NUM_KV); + + final int rowLen = 32; + for (int i = 0; i < NUM_KV; ++i) { + byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i); + byte[] v = TestHFileWriterV2.randomValue(rand); + int cfLen = rand.nextInt(k.length - rowLen + 1); + KeyValue kv = new KeyValue( + k, 0, rowLen, + k, rowLen, cfLen, + k, rowLen + cfLen, k.length - rowLen - cfLen, + rand.nextLong(), + generateKeyType(rand), + v, 0, v.length); + sfw.append(kv); + } + + sfw.close(); + storeFilePath = sfw.getPath(); + } + +} diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java index ba79c82..1ad2ece 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java @@ -20,8 +20,11 @@ package org.apache.hadoop.hbase.io.hfile; import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.io.HeapSize; import java.util.LinkedList; + import junit.framework.TestCase; public class TestCachedBlockQueue extends TestCase { @@ -127,9 +130,13 @@ public class TestCachedBlockQueue extends TestCase { private static class CachedBlock extends org.apache.hadoop.hbase.io.hfile.CachedBlock { - public CachedBlock(long heapSize, String name, long accessTime) { + public CachedBlock(final long heapSize, String name, long accessTime) { super(name, - ByteBuffer.allocate((int)(heapSize - CachedBlock.PER_BLOCK_OVERHEAD)), + new HeapSize(){ + @Override + public long heapSize() { + return ((int)(heapSize - CachedBlock.PER_BLOCK_OVERHEAD)); + }}, accessTime,false); } } diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java new file mode 100644 index 0000000..610c4e7 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java @@ -0,0 +1,227 @@ +/* + * Copyright 2011 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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import static org.junit.Assert.*; + +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.HBaseTestingUtility; +import org.apache.hadoop.hbase.KeyValue; + +@RunWith(Parameterized.class) +public class TestFixedFileTrailer { + + private static final Log LOG = LogFactory.getLog(TestFixedFileTrailer.class); + + /** The number of used fields by version. Indexed by version minus one. */ + private static final int[] NUM_FIELDS_BY_VERSION = new int[] { 8, 13 }; + + private HBaseTestingUtility util = new HBaseTestingUtility(); + private FileSystem fs; + private ByteArrayOutputStream baos = new ByteArrayOutputStream(); + private int version; + + static { + assert NUM_FIELDS_BY_VERSION.length == HFile.MAX_FORMAT_VERSION + - HFile.MIN_FORMAT_VERSION + 1; + } + + public TestFixedFileTrailer(int version) { + this.version = version; + } + + @Parameters + public static Collection getParameters() { + List versionsToTest = new ArrayList(); + for (int v = HFile.MIN_FORMAT_VERSION; v <= HFile.MAX_FORMAT_VERSION; ++v) + versionsToTest.add(new Integer[] { v } ); + return versionsToTest; + } + + @Before + public void setUp() throws IOException { + fs = FileSystem.get(util.getConfiguration()); + } + + @Test + public void testTrailer() throws IOException { + FixedFileTrailer t = new FixedFileTrailer(version); + t.setDataIndexCount(3); + t.setEntryCount(((long) Integer.MAX_VALUE) + 1); + + if (version == 1) { + t.setFileInfoOffset(876); + } + + if (version == 2) { + t.setLastDataBlockOffset(291); + t.setNumDataIndexLevels(3); + t.setComparatorClass(KeyValue.KEY_COMPARATOR.getClass()); + t.setFirstDataBlockOffset(9081723123L); // Completely unrealistic. + t.setUncompressedDataIndexSize(827398717L); // Something random. + } + + t.setLoadOnOpenOffset(128); + t.setMetaIndexCount(7); + + t.setTotalUncompressedBytes(129731987); + + { + DataOutputStream dos = new DataOutputStream(baos); // Limited scope. + t.serialize(dos); + dos.flush(); + assertEquals(dos.size(), FixedFileTrailer.getTrailerSize(version)); + } + + byte[] bytes = baos.toByteArray(); + baos.reset(); + + assertEquals(bytes.length, FixedFileTrailer.getTrailerSize(version)); + + ByteArrayInputStream bais = new ByteArrayInputStream(bytes); + + // Finished writing, trying to read. + { + DataInputStream dis = new DataInputStream(bais); + FixedFileTrailer t2 = new FixedFileTrailer(version); + t2.deserialize(dis); + assertEquals(-1, bais.read()); // Ensure we have read everything. + checkLoadedTrailer(version, t, t2); + } + + // Now check what happens if the trailer is corrupted. + Path trailerPath = new Path(HBaseTestingUtility.getTestDir(), "trailer_" + + version); + + { + for (byte invalidVersion : new byte[] { HFile.MIN_FORMAT_VERSION - 1, + HFile.MAX_FORMAT_VERSION + 1}) { + bytes[bytes.length - 1] = invalidVersion; + writeTrailer(trailerPath, null, bytes); + try { + readTrailer(trailerPath); + fail("Exception expected"); + } catch (IOException ex) { + // Make it easy to debug this. + String msg = ex.getMessage(); + String cleanMsg = msg.replaceAll( + "^(java(\\.[a-zA-Z]+)+:\\s+)?|\\s+\\(.*\\)\\s*$", ""); + assertEquals("Actual exception message is \"" + msg + "\".\n" + + "Cleaned-up message", // will be followed by " expected: ..." + "Invalid HFile version: " + invalidVersion, cleanMsg); + LOG.info("Got an expected exception: " + msg); + } + } + + } + + // Now write the trailer into a file and auto-detect the version. + writeTrailer(trailerPath, t, null); + + FixedFileTrailer t4 = readTrailer(trailerPath); + + checkLoadedTrailer(version, t, t4); + + String trailerStr = t.toString(); + assertEquals("Invalid number of fields in the string representation " + + "of the trailer: " + trailerStr, NUM_FIELDS_BY_VERSION[version - 1], + trailerStr.split(", ").length); + assertEquals(trailerStr, t4.toString()); + } + + private FixedFileTrailer readTrailer(Path trailerPath) throws IOException { + FSDataInputStream fsdis = fs.open(trailerPath); + FixedFileTrailer trailerRead = FixedFileTrailer.readFromStream(fsdis, + fs.getFileStatus(trailerPath).getLen()); + fsdis.close(); + return trailerRead; + } + + private void writeTrailer(Path trailerPath, FixedFileTrailer t, + byte[] useBytesInstead) throws IOException { + assert (t == null) != (useBytesInstead == null); // Expect one non-null. + + FSDataOutputStream fsdos = fs.create(trailerPath); + fsdos.write(135); // to make deserializer's job less trivial + if (useBytesInstead != null) { + fsdos.write(useBytesInstead); + } else { + t.serialize(fsdos); + } + fsdos.close(); + } + + private void checkLoadedTrailer(int version, FixedFileTrailer expected, + FixedFileTrailer loaded) throws IOException { + assertEquals(version, loaded.getVersion()); + assertEquals(expected.getDataIndexCount(), loaded.getDataIndexCount()); + + assertEquals(Math.min(expected.getEntryCount(), + version == 1 ? Integer.MAX_VALUE : Long.MAX_VALUE), + loaded.getEntryCount()); + + if (version == 1) { + assertEquals(expected.getFileInfoOffset(), loaded.getFileInfoOffset()); + } + + if (version == 2) { + assertEquals(expected.getLastDataBlockOffset(), + loaded.getLastDataBlockOffset()); + assertEquals(expected.getNumDataIndexLevels(), + loaded.getNumDataIndexLevels()); + assertEquals(expected.createComparator().getClass().getName(), + loaded.createComparator().getClass().getName()); + assertEquals(expected.getFirstDataBlockOffset(), + loaded.getFirstDataBlockOffset()); + assertTrue( + expected.createComparator() instanceof KeyValue.KeyComparator); + assertEquals(expected.getUncompressedDataIndexSize(), + loaded.getUncompressedDataIndexSize()); + } + + assertEquals(expected.getLoadOnOpenDataOffset(), + loaded.getLoadOnOpenDataOffset()); + assertEquals(expected.getMetaIndexCount(), loaded.getMetaIndexCount()); + + assertEquals(expected.getTotalUncompressedBytes(), + loaded.getTotalUncompressedBytes()); + } + +} diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java index 9d071c7..964419a 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java @@ -33,11 +33,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue.KeyComparator; -import org.apache.hadoop.hbase.io.hfile.HFile.BlockIndex; 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.hbase.util.ClassSize; import org.apache.hadoop.io.Writable; /** @@ -63,9 +61,9 @@ public class TestHFile extends HBaseTestCase { */ public void testEmptyHFile() throws IOException { Path f = new Path(ROOT_DIR, getName()); - Writer w = new Writer(this.fs, f); + Writer w = HFile.getWriterFactory(conf).createWriter(this.fs, f); w.close(); - Reader r = new Reader(fs, f, null, false, false); + Reader r = HFile.createReader(fs, f, null, false, false); r.loadFileInfo(); assertNull(r.getFirstKey()); assertNull(r.getLastKey()); @@ -134,13 +132,13 @@ public class TestHFile extends HBaseTestCase { void basicWithSomeCodec(String codec) throws IOException { Path ncTFile = new Path(ROOT_DIR, "basic.hfile"); FSDataOutputStream fout = createFSOutput(ncTFile); - Writer writer = new Writer(fout, minBlockSize, - Compression.getCompressionAlgorithmByName(codec), null); + Writer writer = HFile.getWriterFactory(conf).createWriter(fout, + minBlockSize, Compression.getCompressionAlgorithmByName(codec), null); LOG.info(writer); writeRecords(writer); fout.close(); FSDataInputStream fin = fs.open(ncTFile); - Reader reader = new Reader(ncTFile, fs.open(ncTFile), + Reader reader = HFile.createReader(ncTFile, fs.open(ncTFile), fs.getFileStatus(ncTFile).getLen(), null, false, false); // Load up the index. reader.loadFileInfo(); @@ -209,13 +207,14 @@ public class TestHFile extends HBaseTestCase { private void metablocks(final String compress) throws Exception { Path mFile = new Path(ROOT_DIR, "meta.hfile"); FSDataOutputStream fout = createFSOutput(mFile); - Writer writer = new Writer(fout, minBlockSize, - Compression.getCompressionAlgorithmByName(compress), null); + Writer writer = HFile.getWriterFactory(conf).createWriter(fout, + minBlockSize, Compression.getCompressionAlgorithmByName(compress), + null); someTestingWithMetaBlock(writer); writer.close(); fout.close(); FSDataInputStream fin = fs.open(mFile); - Reader reader = new Reader(mFile, fs.open(mFile), + Reader reader = HFile.createReader(mFile, fs.open(mFile), this.fs.getFileStatus(mFile).getLen(), null, false, false); reader.loadFileInfo(); // No data -- this should return false. @@ -233,33 +232,35 @@ public class TestHFile extends HBaseTestCase { } public void testNullMetaBlocks() throws Exception { - Path mFile = new Path(ROOT_DIR, "nometa.hfile"); - FSDataOutputStream fout = createFSOutput(mFile); - Writer writer = new Writer(fout, minBlockSize, - Compression.Algorithm.NONE, null); - writer.append("foo".getBytes(), "value".getBytes()); - writer.close(); - fout.close(); - Reader reader = new Reader(fs, mFile, null, false, false); - reader.loadFileInfo(); - assertNull(reader.getMetaBlock("non-existant", false)); + for (Compression.Algorithm compressAlgo : + HBaseTestingUtility.COMPRESSION_ALGORITHMS) { + Path mFile = new Path(ROOT_DIR, "nometa_" + compressAlgo + ".hfile"); + FSDataOutputStream fout = createFSOutput(mFile); + Writer writer = HFile.getWriterFactory(conf).createWriter(fout, + minBlockSize, compressAlgo, null); + writer.append("foo".getBytes(), "value".getBytes()); + writer.close(); + fout.close(); + Reader reader = HFile.createReader(fs, mFile, null, false, false); + reader.loadFileInfo(); + assertNull(reader.getMetaBlock("non-existant", false)); + } } /** * 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.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, (Compression.Algorithm) null, - new KeyComparator() { + Writer writer = HFile.getWriterFactory(conf).createWriter(fout, + minBlockSize, (Compression.Algorithm) null, new KeyComparator() { @Override public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) { @@ -277,27 +278,4 @@ public class TestHFile extends HBaseTestCase { writer.close(); } - /** - * Checks if the HeapSize calculator is within reason - */ - @SuppressWarnings("unchecked") - public void testHeapSizeForBlockIndex() throws IOException{ - Class cl = null; - long expected = 0L; - long actual = 0L; - - cl = BlockIndex.class; - expected = ClassSize.estimateBase(cl, false); - BlockIndex bi = new BlockIndex(Bytes.BYTES_RAWCOMPARATOR); - actual = bi.heapSize(); - //Since the arrays in BlockIndex(byte [][] blockKeys, long [] blockOffsets, - //int [] blockDataSizes) are all null they are not going to show up in the - //HeapSize calculation, so need to remove those array costs from ecpected. - expected -= ClassSize.align(3 * ClassSize.ARRAY); - if(expected != actual) { - ClassSize.estimateBase(cl, true); - assertEquals(expected, actual); - } - } - } diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java new file mode 100644 index 0000000..ddd5ebc --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java @@ -0,0 +1,499 @@ +/* + * Copyright 2011 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 static org.junit.Assert.*; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +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.HBaseTestingUtility; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.io.compress.Compressor; + +import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.*; +import org.junit.Before; +import org.junit.Test; + +public class TestHFileBlock { + + private static final boolean[] BOOLEAN_VALUES = new boolean[] { false, true }; + + private static final Log LOG = LogFactory.getLog(TestHFileBlock.class); + + static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = { + NONE, GZ }; + + // In case we need to temporarily switch some test cases to just test gzip. + static final Compression.Algorithm[] GZIP_ONLY = { GZ }; + + private static final int NUM_TEST_BLOCKS = 1000; + + private static final int NUM_READER_THREADS = 26; + + private static final HBaseTestingUtility TEST_UTIL = + new HBaseTestingUtility(); + private FileSystem fs; + private int uncompressedSizeV1; + + @Before + public void setUp() throws IOException { + fs = FileSystem.get(TEST_UTIL.getConfiguration()); + TEST_UTIL.initTestDir(); + } + + public void writeTestBlockContents(DataOutputStream dos) throws IOException { + // This compresses really well. + for (int i = 0; i < 1000; ++i) + dos.writeInt(i / 100); + } + + public byte[] createTestV1Block(Compression.Algorithm algo) + throws IOException { + Compressor compressor = algo.getCompressor(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + OutputStream os = algo.createCompressionStream(baos, compressor, 0); + DataOutputStream dos = new DataOutputStream(os); + BlockType.META.write(dos); // Let's make this a meta block. + writeTestBlockContents(dos); + uncompressedSizeV1 = dos.size(); + dos.flush(); + algo.returnCompressor(compressor); + return baos.toByteArray(); + } + + private byte[] createTestV2Block(Compression.Algorithm algo) + throws IOException { + final BlockType blockType = BlockType.DATA; + HFileBlock.Writer hbw = new HFileBlock.Writer(algo); + DataOutputStream dos = hbw.startWriting(blockType, false); + writeTestBlockContents(dos); + byte[] headerAndData = hbw.getHeaderAndData(); + assertEquals(1000 * 4, hbw.getUncompressedSizeWithoutHeader()); + hbw.releaseCompressor(); + return headerAndData; + } + + public String createTestBlockStr(Compression.Algorithm algo) + throws IOException { + byte[] testV2Block = createTestV2Block(algo); + int osOffset = HFileBlock.HEADER_SIZE + 9; + if (osOffset < testV2Block.length) { + // Force-set the "OS" field of the gzip header to 3 (Unix) to avoid + // variations across operating systems. + // See http://www.gzip.org/zlib/rfc-gzip.html for gzip format. + testV2Block[osOffset] = 3; + } + return Bytes.toStringBinary(testV2Block); + } + + @Test + public void testNoCompression() throws IOException { + assertEquals(4000 + HFileBlock.HEADER_SIZE, createTestV2Block(NONE).length); + } + + @Test + public void testGzipCompression() throws IOException { + assertEquals( + "DATABLK*\\x00\\x00\\x00:\\x00\\x00\\x0F\\xA0\\xFF\\xFF\\xFF\\xFF" + + "\\xFF\\xFF\\xFF\\xFF" + // gzip-compressed block: http://www.gzip.org/zlib/rfc-gzip.html + + "\\x1F\\x8B" // gzip magic signature + + "\\x08" // Compression method: 8 = "deflate" + + "\\x00" // Flags + + "\\x00\\x00\\x00\\x00" // mtime + + "\\x00" // XFL (extra flags) + // OS (0 = FAT filesystems, 3 = Unix). However, this field + // sometimes gets set to 0 on Linux and Mac, so we reset it to 3. + + "\\x03" + + "\\xED\\xC3\\xC1\\x11\\x00 \\x08\\xC00DD\\xDD\\x7Fa" + + "\\xD6\\xE8\\xA3\\xB9K\\x84`\\x96Q\\xD3\\xA8\\xDB\\xA8e\\xD4c" + + "\\xD46\\xEA5\\xEA3\\xEA7\\xE7\\x00LI\\s\\xA0\\x0F\\x00\\x00", + createTestBlockStr(GZ)); + } + + @Test + public void testReaderV1() throws IOException { + for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) { + for (boolean pread : new boolean[] { false, true }) { + byte[] block = createTestV1Block(algo); + Path path = new Path(HBaseTestingUtility.getTestDir(), "blocks_v1_" + + algo); + LOG.info("Creating temporary file at " + path); + FSDataOutputStream os = fs.create(path); + int totalSize = 0; + int numBlocks = 50; + for (int i = 0; i < numBlocks; ++i) { + os.write(block); + totalSize += block.length; + } + os.close(); + + FSDataInputStream is = fs.open(path); + HFileBlock.FSReader hbr = new HFileBlock.FSReaderV1(is, algo, + totalSize); + HFileBlock b; + int numBlocksRead = 0; + long pos = 0; + while (pos < totalSize) { + b = hbr.readBlockData(pos, block.length, uncompressedSizeV1, pread); + b.sanityCheck(); + pos += block.length; + numBlocksRead++; + } + assertEquals(numBlocks, numBlocksRead); + is.close(); + } + } + } + + @Test + public void testReaderV2() throws IOException { + for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) { + for (boolean pread : new boolean[] { false, true }) { + Path path = new Path(HBaseTestingUtility.getTestDir(), "blocks_v2_" + + algo); + FSDataOutputStream os = fs.create(path); + HFileBlock.Writer hbw = new HFileBlock.Writer(algo); + long totalSize = 0; + for (int blockId = 0; blockId < 2; ++blockId) { + DataOutputStream dos = hbw.startWriting(BlockType.DATA, false); + for (int i = 0; i < 1234; ++i) + dos.writeInt(i); + hbw.writeHeaderAndData(os); + totalSize += hbw.getOnDiskSizeWithHeader(); + } + os.close(); + + FSDataInputStream is = fs.open(path); + HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, algo, + totalSize); + HFileBlock b = hbr.readBlockData(0, -1, -1, pread); + is.close(); + + b.sanityCheck(); + assertEquals(4936, b.getUncompressedSizeWithoutHeader()); + assertEquals(algo == GZ ? 2173 : 4936, b.getOnDiskSizeWithoutHeader()); + String blockStr = b.toString(); + + if (algo == GZ) { + is = fs.open(path); + hbr = new HFileBlock.FSReaderV2(is, algo, totalSize); + b = hbr.readBlockData(0, 2173 + HFileBlock.HEADER_SIZE, -1, pread); + assertEquals(blockStr, b.toString()); + int wrongCompressedSize = 2172; + try { + b = hbr.readBlockData(0, wrongCompressedSize + + HFileBlock.HEADER_SIZE, -1, pread); + fail("Exception expected"); + } catch (IOException ex) { + String expectedPrefix = "On-disk size without header provided is " + + wrongCompressedSize + ", but block header contains " + + b.getOnDiskSizeWithoutHeader() + "."; + assertTrue("Invalid exception message: '" + ex.getMessage() + + "'.\nMessage is expected to start with: '" + expectedPrefix + + "'", ex.getMessage().startsWith(expectedPrefix)); + } + is.close(); + } + } + } + } + + @Test + public void testPreviousOffset() throws IOException { + for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) { + for (boolean pread : BOOLEAN_VALUES) { + for (boolean cacheOnWrite : BOOLEAN_VALUES) { + Random rand = defaultRandom(); + LOG.info("Compression algorithm: " + algo + ", pread=" + pread); + Path path = new Path(HBaseTestingUtility.getTestDir(), "prev_offset"); + List expectedOffsets = new ArrayList(); + List expectedPrevOffsets = new ArrayList(); + List expectedTypes = new ArrayList(); + List expectedContents = cacheOnWrite + ? new ArrayList() : null; + long totalSize = writeBlocks(rand, algo, path, expectedOffsets, + expectedPrevOffsets, expectedTypes, expectedContents, true); + + FSDataInputStream is = fs.open(path); + HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, algo, + totalSize); + long curOffset = 0; + for (int i = 0; i < NUM_TEST_BLOCKS; ++i) { + if (!pread) { + assertEquals(is.getPos(), curOffset + (i == 0 ? 0 : + HFileBlock.HEADER_SIZE)); + } + + assertEquals(expectedOffsets.get(i).longValue(), curOffset); + + LOG.info("Reading block #" + i + " at offset " + curOffset); + HFileBlock b = hbr.readBlockData(curOffset, -1, -1, pread); + LOG.info("Block #" + i + ": " + b); + assertEquals("Invalid block #" + i + "'s type:", + expectedTypes.get(i), b.getBlockType()); + assertEquals("Invalid previous block offset for block " + i + + " of " + "type " + b.getBlockType() + ":", + (long) expectedPrevOffsets.get(i), b.getPrevBlockOffset()); + b.sanityCheck(); + assertEquals(curOffset, b.getOffset()); + + // Now re-load this block knowing the on-disk size. This tests a + // different branch in the loader. + HFileBlock b2 = hbr.readBlockData(curOffset, + b.getOnDiskSizeWithHeader(), -1, pread); + b2.sanityCheck(); + + assertEquals(b.getBlockType(), b2.getBlockType()); + assertEquals(b.getOnDiskSizeWithoutHeader(), + b2.getOnDiskSizeWithoutHeader()); + assertEquals(b.getOnDiskSizeWithHeader(), + b2.getOnDiskSizeWithHeader()); + assertEquals(b.getUncompressedSizeWithoutHeader(), + b2.getUncompressedSizeWithoutHeader()); + assertEquals(b.getPrevBlockOffset(), b2.getPrevBlockOffset()); + assertEquals(curOffset, b2.getOffset()); + + curOffset += b.getOnDiskSizeWithHeader(); + + if (cacheOnWrite) { + // In the cache-on-write mode we store uncompressed bytes so we + // can compare them to what was read by the block reader. + + ByteBuffer bufRead = b.getBufferWithHeader(); + ByteBuffer bufExpected = expectedContents.get(i); + boolean bytesAreCorrect = Bytes.compareTo(bufRead.array(), + bufRead.arrayOffset(), bufRead.limit(), + bufExpected.array(), bufExpected.arrayOffset(), + bufExpected.limit()) == 0; + String wrongBytesMsg = ""; + + if (!bytesAreCorrect) { + // Optimization: only construct an error message in case we + // will need it. + wrongBytesMsg = "Expected bytes in block #" + i + " (algo=" + + algo + ", pread=" + pread + "):\n"; + wrongBytesMsg += Bytes.toStringBinary(bufExpected.array(), + bufExpected.arrayOffset(), Math.min(32, + bufExpected.limit())) + + ", actual:\n" + + Bytes.toStringBinary(bufRead.array(), + bufRead.arrayOffset(), Math.min(32, bufRead.limit())); + } + + assertTrue(wrongBytesMsg, bytesAreCorrect); + } + } + + assertEquals(curOffset, fs.getFileStatus(path).getLen()); + is.close(); + } + } + } + } + + private Random defaultRandom() { + return new Random(189237); + } + + private class BlockReaderThread implements Callable { + private final String clientId; + private final HFileBlock.FSReader hbr; + private final List offsets; + private final List types; + private final long fileSize; + + public BlockReaderThread(String clientId, + HFileBlock.FSReader hbr, List offsets, List types, + long fileSize) { + this.clientId = clientId; + this.offsets = offsets; + this.hbr = hbr; + this.types = types; + this.fileSize = fileSize; + } + + @Override + public Boolean call() throws Exception { + Random rand = new Random(clientId.hashCode()); + long endTime = System.currentTimeMillis() + 10000; + int numBlocksRead = 0; + int numPositionalRead = 0; + int numWithOnDiskSize = 0; + while (System.currentTimeMillis() < endTime) { + int blockId = rand.nextInt(NUM_TEST_BLOCKS); + long offset = offsets.get(blockId); + boolean pread = rand.nextBoolean(); + boolean withOnDiskSize = rand.nextBoolean(); + long expectedSize = + (blockId == NUM_TEST_BLOCKS - 1 ? fileSize + : offsets.get(blockId + 1)) - offset; + + HFileBlock b; + try { + long onDiskSizeArg = withOnDiskSize ? expectedSize : -1; + b = hbr.readBlockData(offset, onDiskSizeArg, -1, pread); + } catch (IOException ex) { + LOG.error("Error in client " + clientId + " trying to read block at " + + offset + ", pread=" + pread + ", withOnDiskSize=" + + withOnDiskSize, ex); + return false; + } + + assertEquals(types.get(blockId), b.getBlockType()); + assertEquals(expectedSize, b.getOnDiskSizeWithHeader()); + assertEquals(offset, b.getOffset()); + + ++numBlocksRead; + if (pread) + ++numPositionalRead; + if (withOnDiskSize) + ++numWithOnDiskSize; + } + LOG.info("Client " + clientId + " successfully read " + numBlocksRead + + " blocks (with pread: " + numPositionalRead + ", with onDiskSize " + + "specified: " + numWithOnDiskSize + ")"); + return true; + } + + } + + @Test + public void testConcurrentReading() throws Exception { + for (Compression.Algorithm compressAlgo : COMPRESSION_ALGORITHMS) { + Path path = + new Path(HBaseTestingUtility.getTestDir(), "concurrent_reading"); + Random rand = defaultRandom(); + List offsets = new ArrayList(); + List types = new ArrayList(); + writeBlocks(rand, compressAlgo, path, offsets, null, types, null, false); + FSDataInputStream is = fs.open(path); + long fileSize = fs.getFileStatus(path).getLen(); + HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, compressAlgo, + fileSize); + + Executor exec = Executors.newFixedThreadPool(NUM_READER_THREADS); + ExecutorCompletionService ecs = + new ExecutorCompletionService(exec); + + for (int i = 0; i < NUM_READER_THREADS; ++i) { + ecs.submit(new BlockReaderThread("reader_" + (char) ('A' + i), hbr, + offsets, types, fileSize)); + } + + for (int i = 0; i < NUM_READER_THREADS; ++i) { + Future result = ecs.take(); + assertTrue(result.get()); + LOG.info(String.valueOf(i + 1) + + " reader threads finished successfully (algo=" + compressAlgo + + ")"); + } + + is.close(); + } + } + + private long writeBlocks(Random rand, Compression.Algorithm compressAlgo, + Path path, List expectedOffsets, List expectedPrevOffsets, + List expectedTypes, List expectedContents, + boolean detailedLogging) throws IOException { + boolean cacheOnWrite = expectedContents != null; + FSDataOutputStream os = fs.create(path); + HFileBlock.Writer hbw = new HFileBlock.Writer(compressAlgo); + Map prevOffsetByType = new HashMap(); + long totalSize = 0; + for (int i = 0; i < NUM_TEST_BLOCKS; ++i) { + int blockTypeOrdinal = rand.nextInt(BlockType.values().length); + BlockType bt = BlockType.values()[blockTypeOrdinal]; + DataOutputStream dos = hbw.startWriting(bt, cacheOnWrite); + for (int j = 0; j < rand.nextInt(500); ++j) { + // This might compress well. + dos.writeShort(i + 1); + dos.writeInt(j + 1); + } + + if (expectedOffsets != null) + expectedOffsets.add(os.getPos()); + + if (expectedPrevOffsets != null) { + Long prevOffset = prevOffsetByType.get(bt); + expectedPrevOffsets.add(prevOffset != null ? prevOffset : -1); + prevOffsetByType.put(bt, os.getPos()); + } + + expectedTypes.add(bt); + + hbw.writeHeaderAndData(os); + totalSize += hbw.getOnDiskSizeWithHeader(); + + if (cacheOnWrite) + expectedContents.add(hbw.getUncompressedBufferWithHeader()); + + if (detailedLogging) { + LOG.info("Writing block #" + i + " of type " + bt + + ", uncompressed size " + hbw.getUncompressedSizeWithoutHeader() + + " at offset " + os.getPos()); + } + } + os.close(); + LOG.info("Created a temporary file at " + path + ", " + + fs.getFileStatus(path).getLen() + " byte, compression=" + + compressAlgo); + return totalSize; + } + + @Test + public void testBlockHeapSize() { + for (int size : new int[] { 100, 256, 12345 }) { + byte[] byteArr = new byte[HFileBlock.HEADER_SIZE + size]; + ByteBuffer buf = ByteBuffer.wrap(byteArr, 0, size); + HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf, + true, -1); + assertEquals(80, HFileBlock.BYTE_BUFFER_HEAP_SIZE); + long expected = ClassSize.align(ClassSize.estimateBase(HFileBlock.class, + true) + + ClassSize.estimateBase(buf.getClass(), true) + + HFileBlock.HEADER_SIZE + size); + assertEquals(expected, block.heapSize()); + } + } + +} diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java new file mode 100644 index 0000000..4dc1367 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java @@ -0,0 +1,602 @@ +/* + * Copyright 2011 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.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; + +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.Path; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexReader; +import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexChunk; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import static org.junit.Assert.*; + +@RunWith(Parameterized.class) +public class TestHFileBlockIndex { + + @Parameters + public static Collection compressionAlgorithms() { + return HBaseTestingUtility.COMPRESSION_ALGORITHMS_PARAMETERIZED; + } + + public TestHFileBlockIndex(Compression.Algorithm compr) { + this.compr = compr; + } + + private static final Log LOG = LogFactory.getLog(TestHFileBlockIndex.class); + + private static final int NUM_DATA_BLOCKS = 1000; + private static final HBaseTestingUtility TEST_UTIL = + new HBaseTestingUtility(); + + private static final int SMALL_BLOCK_SIZE = 4096; + private static final int NUM_KV = 10000; + + private static FileSystem fs; + private Path path; + private Random rand; + private long rootIndexOffset; + private int numRootEntries; + private int numLevels; + private static final List keys = new ArrayList(); + private final Compression.Algorithm compr; + private byte[] firstKeyInFile; + private Configuration conf; + + private static final int[] INDEX_CHUNK_SIZES = { 4096, 512, 384 }; + private static final int[] EXPECTED_NUM_LEVELS = { 2, 3, 4 }; + private static final int[] UNCOMPRESSED_INDEX_SIZES = + { 19187, 21813, 23086 }; + + static { + assert INDEX_CHUNK_SIZES.length == EXPECTED_NUM_LEVELS.length; + assert INDEX_CHUNK_SIZES.length == UNCOMPRESSED_INDEX_SIZES.length; + } + + @Before + public void setUp() throws IOException { + keys.clear(); + rand = new Random(2389757); + firstKeyInFile = null; + conf = TEST_UTIL.getConfiguration(); + + // This test requires at least HFile format version 2. + conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION); + + fs = FileSystem.get(conf); + } + + @Test + public void testBlockIndex() throws IOException { + path = new Path(HBaseTestingUtility.getTestDir(), "block_index_" + compr); + writeWholeIndex(); + readIndex(); + } + + /** + * A wrapper around a block reader which only caches the results of the last + * operation. Not thread-safe. + */ + private static class BlockReaderWrapper implements HFileBlock.BasicReader { + + private HFileBlock.BasicReader realReader; + private long prevOffset; + private long prevOnDiskSize; + private long prevUncompressedSize; + private boolean prevPread; + private HFileBlock prevBlock; + + public int hitCount = 0; + public int missCount = 0; + + public BlockReaderWrapper(HFileBlock.BasicReader realReader) { + this.realReader = realReader; + } + + @Override + public HFileBlock readBlockData(long offset, long onDiskSize, + int uncompressedSize, boolean pread) throws IOException { + if (offset == prevOffset && onDiskSize == prevOnDiskSize && + uncompressedSize == prevUncompressedSize && pread == prevPread) { + hitCount += 1; + return prevBlock; + } + + missCount += 1; + prevBlock = realReader.readBlockData(offset, onDiskSize, + uncompressedSize, pread); + prevOffset = offset; + prevOnDiskSize = onDiskSize; + prevUncompressedSize = uncompressedSize; + prevPread = pread; + + return prevBlock; + } + } + + public void readIndex() throws IOException { + long fileSize = fs.getFileStatus(path).getLen(); + LOG.info("Size of " + path + ": " + fileSize); + + FSDataInputStream istream = fs.open(path); + HFileBlock.FSReader blockReader = new HFileBlock.FSReaderV2(istream, + compr, fs.getFileStatus(path).getLen()); + + BlockReaderWrapper brw = new BlockReaderWrapper(blockReader); + HFileBlockIndex.BlockIndexReader indexReader = + new HFileBlockIndex.BlockIndexReader( + Bytes.BYTES_RAWCOMPARATOR, numLevels, brw); + + indexReader.readRootIndex(blockReader.blockRange(rootIndexOffset, + fileSize).nextBlockAsStream(BlockType.ROOT_INDEX), numRootEntries); + + long prevOffset = -1; + int i = 0; + int expectedHitCount = 0; + int expectedMissCount = 0; + LOG.info("Total number of keys: " + keys.size()); + for (byte[] key : keys) { + assertTrue(key != null); + assertTrue(indexReader != null); + HFileBlock b = indexReader.seekToDataBlock(key, 0, key.length, null); + if (Bytes.BYTES_RAWCOMPARATOR.compare(key, firstKeyInFile) < 0) { + assertTrue(b == null); + ++i; + continue; + } + + String keyStr = "key #" + i + ", " + Bytes.toStringBinary(key); + + assertTrue("seekToDataBlock failed for " + keyStr, b != null); + + if (prevOffset == b.getOffset()) { + assertEquals(++expectedHitCount, brw.hitCount); + } else { + LOG.info("First key in a new block: " + keyStr + ", block offset: " + + b.getOffset() + ")"); + assertTrue(b.getOffset() > prevOffset); + assertEquals(++expectedMissCount, brw.missCount); + prevOffset = b.getOffset(); + } + ++i; + } + + istream.close(); + } + + private void writeWholeIndex() throws IOException { + assertEquals(0, keys.size()); + HFileBlock.Writer hbw = new HFileBlock.Writer(compr); + FSDataOutputStream outputStream = fs.create(path); + HFileBlockIndex.BlockIndexWriter biw = + new HFileBlockIndex.BlockIndexWriter(hbw, null, null); + + for (int i = 0; i < NUM_DATA_BLOCKS; ++i) { + hbw.startWriting(BlockType.DATA, false).write( + String.valueOf(rand.nextInt(1000)).getBytes()); + long blockOffset = outputStream.getPos(); + hbw.writeHeaderAndData(outputStream); + + byte[] firstKey = null; + for (int j = 0; j < 16; ++j) { + byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i * 16 + j); + keys.add(k); + if (j == 8) + firstKey = k; + } + assertTrue(firstKey != null); + if (firstKeyInFile == null) + firstKeyInFile = firstKey; + biw.addEntry(firstKey, blockOffset, hbw.getOnDiskSizeWithHeader()); + + writeInlineBlocks(hbw, outputStream, biw, false); + } + writeInlineBlocks(hbw, outputStream, biw, true); + rootIndexOffset = biw.writeIndexBlocks(outputStream); + outputStream.close(); + + numLevels = biw.getNumLevels(); + numRootEntries = biw.getNumRootEntries(); + + LOG.info("Index written: numLevels=" + numLevels + ", numRootEntries=" + + numRootEntries + ", rootIndexOffset=" + rootIndexOffset); + } + + private void writeInlineBlocks(HFileBlock.Writer hbw, + FSDataOutputStream outputStream, HFileBlockIndex.BlockIndexWriter biw, + boolean isClosing) throws IOException { + while (biw.shouldWriteBlock(isClosing)) { + long offset = outputStream.getPos(); + biw.writeInlineBlock(hbw.startWriting(biw.getInlineBlockType(), false)); + hbw.writeHeaderAndData(outputStream); + biw.blockWritten(offset, hbw.getOnDiskSizeWithHeader(), + hbw.getUncompressedSizeWithoutHeader()); + LOG.info("Wrote an inline index block at " + offset + ", size " + + hbw.getOnDiskSizeWithHeader()); + } + } + + private static final long getDummyFileOffset(int i) { + return i * 185 + 379; + } + + private static final int getDummyOnDiskSize(int i) { + return i * i * 37 + i * 19 + 13; + } + + @Test + public void testSecondaryIndexBinarySearch() throws IOException { + int numTotalKeys = 99; + assertTrue(numTotalKeys % 2 == 1); // Ensure no one made this even. + + // We only add odd-index keys into the array that we will binary-search. + int numSearchedKeys = (numTotalKeys - 1) / 2; + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos); + + dos.writeInt(numSearchedKeys); + int curAllEntriesSize = 0; + int numEntriesAdded = 0; + + // Only odd-index elements of this array are used to keep the secondary + // index entries of the corresponding keys. + int secondaryIndexEntries[] = new int[numTotalKeys]; + + for (int i = 0; i < numTotalKeys; ++i) { + byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i * 2); + keys.add(k); + String msgPrefix = "Key #" + i + " (" + Bytes.toStringBinary(k) + "): "; + StringBuilder padding = new StringBuilder(); + while (msgPrefix.length() + padding.length() < 70) + padding.append(' '); + msgPrefix += padding; + if (i % 2 == 1) { + dos.writeInt(curAllEntriesSize); + secondaryIndexEntries[i] = curAllEntriesSize; + LOG.info(msgPrefix + "secondary index entry #" + ((i - 1) / 2) + + ", offset " + curAllEntriesSize); + curAllEntriesSize += k.length + + HFileBlockIndex.SECONDARY_INDEX_ENTRY_OVERHEAD; + ++numEntriesAdded; + } else { + secondaryIndexEntries[i] = -1; + LOG.info(msgPrefix + "not in the searched array"); + } + } + + // Make sure the keys are increasing. + for (int i = 0; i < keys.size() - 1; ++i) + assertTrue(Bytes.BYTES_RAWCOMPARATOR.compare(keys.get(i), + keys.get(i + 1)) < 0); + + dos.writeInt(curAllEntriesSize); + assertEquals(numSearchedKeys, numEntriesAdded); + int secondaryIndexOffset = dos.size(); + assertEquals(Bytes.SIZEOF_INT * (numSearchedKeys + 2), + secondaryIndexOffset); + + for (int i = 1; i <= numTotalKeys - 1; i += 2) { + assertEquals(dos.size(), + secondaryIndexOffset + secondaryIndexEntries[i]); + long dummyFileOffset = getDummyFileOffset(i); + int dummyOnDiskSize = getDummyOnDiskSize(i); + LOG.debug("Storing file offset=" + dummyFileOffset + " and onDiskSize=" + + dummyOnDiskSize + " at offset " + dos.size()); + dos.writeLong(dummyFileOffset); + dos.writeInt(dummyOnDiskSize); + LOG.debug("Stored key " + ((i - 1) / 2) +" at offset " + dos.size()); + dos.write(keys.get(i)); + } + + dos.writeInt(curAllEntriesSize); + + ByteBuffer nonRootIndex = ByteBuffer.wrap(baos.toByteArray()); + for (int i = 0; i < numTotalKeys; ++i) { + byte[] searchKey = keys.get(i); + byte[] arrayHoldingKey = new byte[searchKey.length + + searchKey.length / 2]; + + // To make things a bit more interesting, store the key we are looking + // for at a non-zero offset in a new array. + System.arraycopy(searchKey, 0, arrayHoldingKey, searchKey.length / 2, + searchKey.length); + + int searchResult = BlockIndexReader.binarySearchNonRootIndex( + arrayHoldingKey, searchKey.length / 2, searchKey.length, nonRootIndex, + Bytes.BYTES_RAWCOMPARATOR); + String lookupFailureMsg = "Failed to look up key #" + i + " (" + + Bytes.toStringBinary(searchKey) + ")"; + + int expectedResult; + int referenceItem; + + if (i % 2 == 1) { + // This key is in the array we search as the element (i - 1) / 2. Make + // sure we find it. + expectedResult = (i - 1) / 2; + referenceItem = i; + } else { + // This key is not in the array but between two elements on the array, + // in the beginning, or in the end. The result should be the previous + // key in the searched array, or -1 for i = 0. + expectedResult = i / 2 - 1; + referenceItem = i - 1; + } + + assertEquals(lookupFailureMsg, expectedResult, searchResult); + + // Now test we can get the offset and the on-disk-size using a + // higher-level API function.s + boolean locateBlockResult = + BlockIndexReader.locateNonRootIndexEntry(nonRootIndex, arrayHoldingKey, + searchKey.length / 2, searchKey.length, Bytes.BYTES_RAWCOMPARATOR); + + if (i == 0) { + assertFalse(locateBlockResult); + } else { + assertTrue(locateBlockResult); + String errorMsg = "i=" + i + ", position=" + nonRootIndex.position(); + assertEquals(errorMsg, getDummyFileOffset(referenceItem), + nonRootIndex.getLong()); + assertEquals(errorMsg, getDummyOnDiskSize(referenceItem), + nonRootIndex.getInt()); + } + } + + } + + @Test + public void testBlockIndexChunk() throws IOException { + BlockIndexChunk c = new BlockIndexChunk(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + int N = 1000; + int[] numSubEntriesAt = new int[N]; + int numSubEntries = 0; + for (int i = 0; i < N; ++i) { + baos.reset(); + DataOutputStream dos = new DataOutputStream(baos); + c.writeNonRoot(dos); + assertEquals(c.getNonRootSize(), dos.size()); + + baos.reset(); + dos = new DataOutputStream(baos); + c.writeRoot(dos); + assertEquals(c.getRootSize(), dos.size()); + + byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i); + numSubEntries += rand.nextInt(5) + 1; + keys.add(k); + c.add(k, getDummyFileOffset(i), getDummyOnDiskSize(i), numSubEntries); + } + + // Test the ability to look up the entry that contains a particular + // deeper-level index block's entry ("sub-entry"), assuming a global + // 0-based ordering of sub-entries. This is needed for mid-key calculation. + for (int i = 0; i < N; ++i) { + for (int j = i == 0 ? 0 : numSubEntriesAt[i - 1]; + j < numSubEntriesAt[i]; + ++j) { + assertEquals(i, c.getEntryBySubEntry(j)); + } + } + } + + /** Checks if the HeapSize calculator is within reason */ + @Test + public void testHeapSizeForBlockIndex() throws IOException { + Class cl = + HFileBlockIndex.BlockIndexReader.class; + long expected = ClassSize.estimateBase(cl, false); + + HFileBlockIndex.BlockIndexReader bi = + new HFileBlockIndex.BlockIndexReader(Bytes.BYTES_RAWCOMPARATOR, 1); + long actual = bi.heapSize(); + + // Since the arrays in BlockIndex(byte [][] blockKeys, long [] blockOffsets, + // int [] blockDataSizes) are all null they are not going to show up in the + // HeapSize calculation, so need to remove those array costs from expected. + expected -= ClassSize.align(3 * ClassSize.ARRAY); + + if (expected != actual) { + ClassSize.estimateBase(cl, true); + assertEquals(expected, actual); + } + } + + /** + * Testing block index through the HFile writer/reader APIs. Allows to test + * setting index block size through configuration, intermediate-level index + * blocks, and caching index blocks on write. + * + * @throws IOException + */ + @Test + public void testHFileWriterAndReader() throws IOException { + Path hfilePath = new Path(HBaseTestingUtility.getTestDir(), + "hfile_for_block_index"); + BlockCache blockCache = StoreFile.getBlockCache(conf); + + for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; ++testI) { + int indexBlockSize = INDEX_CHUNK_SIZES[testI]; + int expectedNumLevels = EXPECTED_NUM_LEVELS[testI]; + LOG.info("Index block size: " + indexBlockSize + ", compression: " + + compr); + // Evict all blocks that were cached-on-write by the previous invocation. + blockCache.evictBlocksByPrefix(hfilePath.getName() + + HFile.CACHE_KEY_SEPARATOR); + + conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, indexBlockSize); + Set keyStrSet = new HashSet(); + byte[][] keys = new byte[NUM_KV][]; + byte[][] values = new byte[NUM_KV][]; + + // Write the HFile + { + HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(fs, + hfilePath, SMALL_BLOCK_SIZE, compr, KeyValue.KEY_COMPARATOR); + Random rand = new Random(19231737); + + for (int i = 0; i < NUM_KV; ++i) { + byte[] row = TestHFileWriterV2.randomOrderedKey(rand, i); + + // Key will be interpreted by KeyValue.KEY_COMPARATOR + byte[] k = KeyValue.createFirstOnRow(row, 0, row.length, row, 0, 0, + row, 0, 0).getKey(); + + byte[] v = TestHFileWriterV2.randomValue(rand); + writer.append(k, v); + keys[i] = k; + values[i] = v; + keyStrSet.add(Bytes.toStringBinary(k)); + + if (i > 0) { + assertTrue(KeyValue.KEY_COMPARATOR.compare(keys[i - 1], + keys[i]) < 0); + } + } + + writer.close(); + } + + // Read the HFile + HFile.Reader reader = HFile.createReader(fs, hfilePath, blockCache, + false, true); + assertEquals(expectedNumLevels, + reader.getTrailer().getNumDataIndexLevels()); + + assertTrue(Bytes.equals(keys[0], reader.getFirstKey())); + assertTrue(Bytes.equals(keys[NUM_KV - 1], reader.getLastKey())); + LOG.info("Last key: " + Bytes.toStringBinary(keys[NUM_KV - 1])); + + for (boolean pread : new boolean[] { false, true }) { + HFileScanner scanner = reader.getScanner(true, pread); + for (int i = 0; i < NUM_KV; ++i) { + checkSeekTo(keys, scanner, i); + checkKeyValue("i=" + i, keys[i], values[i], scanner.getKey(), + scanner.getValue()); + } + assertTrue(scanner.seekTo()); + for (int i = NUM_KV - 1; i >= 0; --i) { + checkSeekTo(keys, scanner, i); + checkKeyValue("i=" + i, keys[i], values[i], scanner.getKey(), + scanner.getValue()); + } + } + + // Manually compute the mid-key and validate it. + HFileReaderV2 reader2 = (HFileReaderV2) reader; + HFileBlock.FSReader fsReader = reader2.getUncachedBlockReader(); + + HFileBlock.BlockIterator iter = fsReader.blockRange(0, + reader.getTrailer().getLoadOnOpenDataOffset()); + HFileBlock block; + List blockKeys = new ArrayList(); + while ((block = iter.nextBlock()) != null) { + if (block.getBlockType() != BlockType.LEAF_INDEX) + return; + ByteBuffer b = block.getBufferReadOnly(); + int n = b.getInt(); + // One int for the number of items, and n + 1 for the secondary index. + int entriesOffset = Bytes.SIZEOF_INT * (n + 2); + + // Get all the keys from the leaf index block. S + for (int i = 0; i < n; ++i) { + int keyRelOffset = b.getInt(Bytes.SIZEOF_INT * (i + 1)); + int nextKeyRelOffset = b.getInt(Bytes.SIZEOF_INT * (i + 2)); + int keyLen = nextKeyRelOffset - keyRelOffset; + int keyOffset = b.arrayOffset() + entriesOffset + keyRelOffset + + HFileBlockIndex.SECONDARY_INDEX_ENTRY_OVERHEAD; + byte[] blockKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset + + keyLen); + String blockKeyStr = Bytes.toString(blockKey); + blockKeys.add(blockKey); + + // If the first key of the block is not among the keys written, we + // are not parsing the non-root index block format correctly. + assertTrue("Invalid block key from leaf-level block: " + blockKeyStr, + keyStrSet.contains(blockKeyStr)); + } + } + + // Validate the mid-key. + assertEquals( + Bytes.toStringBinary(blockKeys.get((blockKeys.size() - 1) / 2)), + Bytes.toStringBinary(reader.midkey())); + + assertEquals(UNCOMPRESSED_INDEX_SIZES[testI], + reader.getTrailer().getUncompressedDataIndexSize()); + + reader.close(); + } + } + + private void checkSeekTo(byte[][] keys, HFileScanner scanner, int i) + throws IOException { + assertEquals("Failed to seek to key #" + i + " (" + + Bytes.toStringBinary(keys[i]) + ")", 0, scanner.seekTo(keys[i])); + } + + private void assertArrayEqualsBuffer(String msgPrefix, byte[] arr, + ByteBuffer buf) { + assertEquals(msgPrefix + ": expected " + Bytes.toStringBinary(arr) + + ", actual " + Bytes.toStringBinary(buf), 0, Bytes.compareTo(arr, 0, + arr.length, buf.array(), buf.arrayOffset(), buf.limit())); + } + + /** Check a key/value pair after it was read by the reader */ + private void checkKeyValue(String msgPrefix, byte[] expectedKey, + byte[] expectedValue, ByteBuffer keyRead, ByteBuffer valueRead) { + if (!msgPrefix.isEmpty()) + msgPrefix += ". "; + + assertArrayEqualsBuffer(msgPrefix + "Invalid key", expectedKey, keyRead); + assertArrayEqualsBuffer(msgPrefix + "Invalid value", expectedValue, + valueRead); + } + +} diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java index 0b3e2c9..247fb7a 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java @@ -157,8 +157,8 @@ public class TestHFilePerformance extends TestCase { if ("HFile".equals(fileType)){ System.out.println("HFile write method: "); - HFile.Writer writer = - new HFile.Writer(fout, minBlockSize, codecName, null); + HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(fout, + minBlockSize, codecName, null); // Writing value in one shot. for (long l=0 ; l keys = new ArrayList(); + List values = new ArrayList(); + + for (int i = 0; i < ENTRY_COUNT; ++i) { + byte[] keyBytes = randomOrderedKey(rand, i); + + // A random-length random value. + byte[] valueBytes = randomValue(rand); + writer.append(keyBytes, valueBytes); + + totalKeyLength += keyBytes.length; + totalValueLength += valueBytes.length; + + keys.add(keyBytes); + values.add(valueBytes); + } + + // Add in an arbitrary order. They will be sorted lexicographically by + // the key. + writer.appendMetaBlock("CAPITAL_OF_USA", new Text("Washington, D.C.")); + writer.appendMetaBlock("CAPITAL_OF_RUSSIA", new Text("Moscow")); + writer.appendMetaBlock("CAPITAL_OF_FRANCE", new Text("Paris")); + + writer.close(); + + FSDataInputStream fsdis = fs.open(hfilePath); + + // A "manual" version of a new-format HFile reader. This unit test was + // written before the V2 reader was fully implemented. + + long fileSize = fs.getFileStatus(hfilePath).getLen(); + FixedFileTrailer trailer = + FixedFileTrailer.readFromStream(fsdis, fileSize); + + assertEquals(2, trailer.getVersion()); + assertEquals(ENTRY_COUNT, trailer.getEntryCount()); + + HFileBlock.FSReader blockReader = + new HFileBlock.FSReaderV2(fsdis, COMPRESS_ALGO, fileSize); + + // Counters for the number of key/value pairs and the number of blocks + int entriesRead = 0; + int blocksRead = 0; + + // Scan blocks the way the reader would scan them + fsdis.seek(0); + long curBlockPos = 0; + while (curBlockPos <= trailer.getLastDataBlockOffset()) { + HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false); + assertEquals(BlockType.DATA, block.getBlockType()); + ByteBuffer buf = block.getBufferWithoutHeader(); + while (buf.hasRemaining()) { + int keyLen = buf.getInt(); + int valueLen = buf.getInt(); + + byte[] key = new byte[keyLen]; + buf.get(key); + + byte[] value = new byte[valueLen]; + buf.get(value); + + // A brute-force check to see that all keys and values are correct. + assertTrue(Bytes.compareTo(key, keys.get(entriesRead)) == 0); + assertTrue(Bytes.compareTo(value, values.get(entriesRead)) == 0); + + ++entriesRead; + } + ++blocksRead; + curBlockPos += block.getOnDiskSizeWithHeader(); + } + LOG.info("Finished reading: entries=" + entriesRead + ", blocksRead=" + + blocksRead); + assertEquals(ENTRY_COUNT, entriesRead); + + // Meta blocks. We can scan until the load-on-open data offset (which is + // the root block index offset in version 2) because we are not testing + // intermediate-level index blocks here. + + int metaCounter = 0; + while (fsdis.getPos() < trailer.getLoadOnOpenDataOffset()) { + LOG.info("Current offset: " + fsdis.getPos() + ", scanning until " + + trailer.getLoadOnOpenDataOffset()); + HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false); + assertEquals(BlockType.META, block.getBlockType()); + Text t = new Text(); + block.readInto(t); + Text expectedText = + (metaCounter == 0 ? new Text("Paris") : metaCounter == 1 ? new Text( + "Moscow") : new Text("Washington, D.C.")); + assertEquals(expectedText, t); + LOG.info("Read meta block data: " + t); + ++metaCounter; + curBlockPos += block.getOnDiskSizeWithHeader(); + } + + fsdis.close(); + } + + // Static stuff used by various HFile v2 unit tests + + private static final String COLUMN_FAMILY_NAME = "_-myColumnFamily-_"; + private static final int MIN_ROW_OR_QUALIFIER_LENGTH = 64; + private static final int MAX_ROW_OR_QUALIFIER_LENGTH = 128; + + /** + * Generates a random key that is guaranteed to increase as the given index i + * increases. The result consists of a prefix, which is a deterministic + * increasing function of i, and a random suffix. + * + * @param rand + * random number generator to use + * @param i + * @return + */ + public static byte[] randomOrderedKey(Random rand, int i) { + StringBuilder k = new StringBuilder(); + + // The fixed-length lexicographically increasing part of the key. + for (int bitIndex = 31; bitIndex >= 0; --bitIndex) { + if ((i & (1 << bitIndex)) == 0) + k.append("a"); + else + k.append("b"); + } + + // A random-length random suffix of the key. + for (int j = 0; j < rand.nextInt(50); ++j) + k.append(randomReadableChar(rand)); + + byte[] keyBytes = k.toString().getBytes(); + return keyBytes; + } + + public static byte[] randomValue(Random rand) { + StringBuilder v = new StringBuilder(); + for (int j = 0; j < 1 + rand.nextInt(2000); ++j) { + v.append((char) (32 + rand.nextInt(95))); + } + + byte[] valueBytes = v.toString().getBytes(); + return valueBytes; + } + + public static final char randomReadableChar(Random rand) { + int i = rand.nextInt(26 * 2 + 10 + 1); + if (i < 26) + return (char) ('A' + i); + i -= 26; + + if (i < 26) + return (char) ('a' + i); + i -= 26; + + if (i < 10) + return (char) ('0' + i); + i -= 10; + + assert i == 0; + return '_'; + } + + public static byte[] randomRowOrQualifier(Random rand) { + StringBuilder field = new StringBuilder(); + int fieldLen = MIN_ROW_OR_QUALIFIER_LENGTH + + rand.nextInt(MAX_ROW_OR_QUALIFIER_LENGTH + - MIN_ROW_OR_QUALIFIER_LENGTH + 1); + for (int i = 0; i < fieldLen; ++i) + field.append(randomReadableChar(rand)); + return field.toString().getBytes(); + } + + public static KeyValue randomKeyValue(Random rand) { + return new KeyValue(randomRowOrQualifier(rand), + COLUMN_FAMILY_NAME.getBytes(), randomRowOrQualifier(rand), + randomValue(rand)); + } + +} diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java index 2713dc1..f0a9832 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java @@ -19,7 +19,6 @@ */ package org.apache.hadoop.hbase.io.hfile; -import java.nio.ByteBuffer; import java.util.Random; import org.apache.hadoop.hbase.io.HeapSize; @@ -43,11 +42,11 @@ public class TestLruBlockCache extends TestCase { LruBlockCache cache = new LruBlockCache(maxSize,blockSize); - Block [] blocks = generateFixedBlocks(10, blockSize, "block"); + CachedItem [] blocks = generateFixedBlocks(10, blockSize, "block"); // Add all the blocks - for(Block block : blocks) { - cache.cacheBlock(block.blockName, block.buf); + for (CachedItem block : blocks) { + cache.cacheBlock(block.blockName, block); } // Let the eviction run @@ -70,35 +69,35 @@ public class TestLruBlockCache extends TestCase { LruBlockCache cache = new LruBlockCache(maxSize, blockSize); - Block [] blocks = generateRandomBlocks(100, blockSize); + CachedItem [] blocks = generateRandomBlocks(100, blockSize); long expectedCacheSize = cache.heapSize(); // Confirm empty - for(Block block : blocks) { + for (CachedItem block : blocks) { assertTrue(cache.getBlock(block.blockName, true) == null); } // Add blocks - for(Block block : blocks) { - cache.cacheBlock(block.blockName, block.buf); - expectedCacheSize += block.heapSize(); + for (CachedItem block : blocks) { + cache.cacheBlock(block.blockName, block); + expectedCacheSize += block.cacheBlockHeapSize(); } // Verify correctly calculated cache heap size assertEquals(expectedCacheSize, cache.heapSize()); // Check if all blocks are properly cached and retrieved - for(Block block : blocks) { - ByteBuffer buf = cache.getBlock(block.blockName, true); + for (CachedItem block : blocks) { + HeapSize buf = cache.getBlock(block.blockName, true); assertTrue(buf != null); - assertEquals(buf.capacity(), block.buf.capacity()); + assertEquals(buf.heapSize(), block.heapSize()); } // Re-add same blocks and ensure nothing has changed - for(Block block : blocks) { + for (CachedItem block : blocks) { try { - cache.cacheBlock(block.blockName, block.buf); + cache.cacheBlock(block.blockName, block); assertTrue("Cache should not allow re-caching a block", false); } catch(RuntimeException re) { // expected @@ -109,10 +108,10 @@ public class TestLruBlockCache extends TestCase { assertEquals(expectedCacheSize, cache.heapSize()); // Check if all blocks are properly cached and retrieved - for(Block block : blocks) { - ByteBuffer buf = cache.getBlock(block.blockName, true); + for (CachedItem block : blocks) { + HeapSize buf = cache.getBlock(block.blockName, true); assertTrue(buf != null); - assertEquals(buf.capacity(), block.buf.capacity()); + assertEquals(buf.heapSize(), block.heapSize()); } // Expect no evictions @@ -129,14 +128,14 @@ public class TestLruBlockCache extends TestCase { LruBlockCache cache = new LruBlockCache(maxSize,blockSize,false); - Block [] blocks = generateFixedBlocks(10, blockSize, "block"); + CachedItem [] blocks = generateFixedBlocks(10, blockSize, "block"); long expectedCacheSize = cache.heapSize(); // Add all the blocks - for(Block block : blocks) { - cache.cacheBlock(block.blockName, block.buf); - expectedCacheSize += block.heapSize(); + for (CachedItem block : blocks) { + cache.cacheBlock(block.blockName, block); + expectedCacheSize += block.cacheBlockHeapSize(); } // A single eviction run should have occurred @@ -158,7 +157,7 @@ public class TestLruBlockCache extends TestCase { assertTrue(cache.getBlock(blocks[1].blockName, true) == null); for(int i=2;i finfo = rd.loadFileInfo(); byte[] range = finfo.get("TIMERANGE".getBytes()); @@ -578,6 +578,9 @@ public class TestHFileOutputFormat { try { // partial map red setup to get an operational writer for testing + // We turn off the sequence file compression, because DefaultCodec + // pollutes the GZip codec pool with an incompatible compressor. + conf.set("io.seqfile.compression.type", "NONE"); Job job = new Job(conf, "testLocalMRIncrementalLoad"); setupRandomGeneratorMapper(job); HFileOutputFormat.configureIncrementalLoad(job, table); @@ -607,7 +610,8 @@ public class TestHFileOutputFormat { // verify that the compression on this file matches the configured // compression Path dataFilePath = fileSystem.listStatus(f.getPath())[0].getPath(); - Reader reader = new HFile.Reader(fileSystem, dataFilePath, null, false, true); + Reader reader = HFile.createReader(fileSystem, dataFilePath, null, + false, true); reader.loadFileInfo(); assertEquals("Incorrect compression used for column family " + familyStr + "(reader: " + reader + ")", diff --git a/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java b/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java index 64a6930..caed95d 100644 --- a/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java +++ b/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java @@ -101,8 +101,8 @@ public class TestLoadIncrementalHFiles { for (byte[][] range : hfileRanges) { byte[] from = range[0]; byte[] to = range[1]; - createHFile(fs, new Path(familyDir, "hfile_" + hfileIdx++), - FAMILY, QUALIFIER, from, to, 1000); + createHFile(util.getConfiguration(), fs, new Path(familyDir, "hfile_" + + hfileIdx++), FAMILY, QUALIFIER, from, to, 1000); } int expectedRows = hfileIdx * 1000; @@ -132,7 +132,7 @@ public class TestLoadIncrementalHFiles { FileSystem fs = util.getTestFileSystem(); Path testIn = new Path(dir, "testhfile"); HColumnDescriptor familyDesc = new HColumnDescriptor(FAMILY); - createHFile(fs, testIn, FAMILY, QUALIFIER, + createHFile(util.getConfiguration(), fs, testIn, FAMILY, QUALIFIER, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), 1000); Path bottomOut = new Path(dir, "bottom.out"); @@ -151,7 +151,7 @@ public class TestLoadIncrementalHFiles { private int verifyHFile(Path p) throws IOException { Configuration conf = util.getConfiguration(); - HFile.Reader reader = new HFile.Reader( + HFile.Reader reader = HFile.createReader( p.getFileSystem(conf), p, null, false, false); reader.loadFileInfo(); HFileScanner scanner = reader.getScanner(false, false); @@ -171,11 +171,13 @@ public class TestLoadIncrementalHFiles { * TODO put me in an HFileTestUtil or something? */ static void createHFile( + Configuration conf, FileSystem fs, Path path, byte[] family, byte[] qualifier, byte[] startKey, byte[] endKey, int numRows) throws IOException { - HFile.Writer writer = new HFile.Writer(fs, path, BLOCKSIZE, COMPRESSION, + HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(fs, path, + BLOCKSIZE, COMPRESSION, KeyValue.KEY_COMPARATOR); long now = System.currentTimeMillis(); try { diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java new file mode 100644 index 0000000..48e9163 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java @@ -0,0 +1,353 @@ +/* + * 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 static org.junit.Assert.*; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.TreeSet; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.hfile.BlockCache; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2; +import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType; +import org.apache.hadoop.hbase.util.BloomFilterFactory; +import org.apache.hadoop.hbase.util.ByteBloomFilter; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CompoundBloomFilter; +import org.apache.hadoop.hbase.util.CompoundBloomFilterBase; +import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter; +import org.junit.Before; +import org.junit.Test; + +/** + * Tests writing Bloom filter blocks in the same part of the file as data + * blocks. + */ +public class TestCompoundBloomFilter { + + private static final HBaseTestingUtility TEST_UTIL = + new HBaseTestingUtility(); + + private static final Log LOG = LogFactory.getLog( + TestCompoundBloomFilter.class); + + private static final int NUM_TESTS = 9; + private static final BloomType BLOOM_TYPES[] = { BloomType.ROW, + BloomType.ROW, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW, + BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROWCOL, BloomType.ROW }; + + private static final int NUM_KV[]; + static { + final int N = 10000; // Only used in initialization. + NUM_KV = new int[] { 21870, N, N, N, N, 1000, N, 7500, 7500}; + assert NUM_KV.length == NUM_TESTS; + } + + private static final int BLOCK_SIZES[]; + static { + final int blkSize = 65536; + BLOCK_SIZES = new int[] { 512, 1000, blkSize, blkSize, blkSize, 128, 300, + blkSize, blkSize }; + assert BLOCK_SIZES.length == NUM_TESTS; + } + + /** + * Be careful not to specify too high a Bloom filter block size, otherwise + * there will only be one oversized chunk and the observed false positive + * rate will be too low. + */ + private static final int BLOOM_BLOCK_SIZES[] = { 1000, 4096, 4096, 4096, + 8192, 128, 1024, 600, 600 }; + static { assert BLOOM_BLOCK_SIZES.length == NUM_TESTS; } + + private static final double TARGET_ERROR_RATES[] = { 0.025, 0.01, 0.015, + 0.01, 0.03, 0.01, 0.01, 0.07, 0.07 }; + static { assert TARGET_ERROR_RATES.length == NUM_TESTS; } + + /** A false positive rate that is obviously too high. */ + private static final double TOO_HIGH_ERROR_RATE; + static { + double m = 0; + for (double errorRate : TARGET_ERROR_RATES) + m = Math.max(m, errorRate); + TOO_HIGH_ERROR_RATE = m + 0.03; + } + + private static Configuration conf; + private FileSystem fs; + private BlockCache blockCache; + + /** A message of the form "in test#:" to include in logging. */ + private String testIdMsg; + + private static final int GENERATION_SEED = 2319; + private static final int EVALUATION_SEED = 135; + + @Before + public void setUp() throws IOException { + conf = TEST_UTIL.getConfiguration(); + + // This test requires the most recent HFile format (i.e. v2). + conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION); + + fs = FileSystem.get(conf); + + blockCache = StoreFile.getBlockCache(conf); + assertNotNull(blockCache); + } + + private List createSortedKeyValues(Random rand, int n) { + List kvList = new ArrayList(n); + for (int i = 0; i < n; ++i) + kvList.add(TestHFileWriterV2.randomKeyValue(rand)); + Collections.sort(kvList, KeyValue.COMPARATOR); + return kvList; + } + + @Test + public void testCompoundBloomFilter() throws IOException { + conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true); + for (int t = 0; t < NUM_TESTS; ++t) { + conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, + (float) TARGET_ERROR_RATES[t]); + + testIdMsg = "in test #" + t + ":"; + Random generationRand = new Random(GENERATION_SEED); + List kvs = createSortedKeyValues(generationRand, NUM_KV[t]); + BloomType bt = BLOOM_TYPES[t]; + Path sfPath = writeStoreFile(t, bt, kvs); + readStoreFile(t, bt, kvs, sfPath); + } + } + + /** + * Validates the false positive ratio by computing its z-value and comparing + * it to the provided threshold. + * + * @param falsePosRate experimental positive rate + * @param nTrials the number of calls to + * {@link StoreFile.Reader#shouldSeek(Scan, java.util.SortedSet)}. + * @param zValueBoundary z-value boundary, positive for an upper bound and + * negative for a lower bound + * @param cbf the compound Bloom filter we are using + * @param additionalMsg additional message to include in log output and + * assertion failures + */ + private void validateFalsePosRate(double falsePosRate, int nTrials, + double zValueBoundary, CompoundBloomFilter cbf, String additionalMsg) { + double p = BloomFilterFactory.getErrorRate(conf); + double zValue = (falsePosRate - p) / Math.sqrt(p * (1 - p) / nTrials); + + String assortedStatsStr = " (targetErrorRate=" + p + ", falsePosRate=" + + falsePosRate + ", nTrials=" + nTrials + ")"; + LOG.info("z-value is " + zValue + assortedStatsStr); + + boolean isUpperBound = zValueBoundary > 0; + + if (isUpperBound && zValue > zValueBoundary || + !isUpperBound && zValue < zValueBoundary) { + String errorMsg = "False positive rate z-value " + zValue + " is " + + (isUpperBound ? "higher" : "lower") + " than " + zValueBoundary + + assortedStatsStr + ". Per-chunk stats:\n" + + cbf.formatTestingStats(); + fail(errorMsg + additionalMsg); + } + } + + private void readStoreFile(int t, BloomType bt, List kvs, + Path sfPath) throws IOException { + StoreFile sf = new StoreFile(fs, sfPath, true, conf, bt, false); + StoreFile.Reader r = sf.createReader(); + final boolean pread = true; // does not really matter + StoreFileScanner scanner = r.getStoreFileScanner(true, pread); + + { + // Test for false negatives (not allowed). + int numChecked = 0; + for (KeyValue kv : kvs) { + byte[] row = kv.getRow(); + boolean present = isInBloom(scanner, row, kv.getQualifier()); + assertTrue(testIdMsg + " Bloom filter false negative on row " + + Bytes.toStringBinary(row) + " after " + numChecked + + " successful checks", present); + ++numChecked; + } + } + + // Test for false positives (some percentage allowed). We test in two modes: + // "fake lookup" which ignores the key distribution, and production mode. + for (boolean fakeLookupEnabled : new boolean[] { true, false }) { + ByteBloomFilter.setFakeLookupMode(fakeLookupEnabled); + try { + String fakeLookupModeStr = ", fake lookup is " + (fakeLookupEnabled ? + "enabled" : "disabled"); + CompoundBloomFilter cbf = (CompoundBloomFilter) r.getBloomFilter(); + cbf.enableTestingStats(); + int numFalsePos = 0; + Random rand = new Random(EVALUATION_SEED); + int nTrials = NUM_KV[t] * 10; + for (int i = 0; i < nTrials; ++i) { + byte[] query = TestHFileWriterV2.randomRowOrQualifier(rand); + if (isInBloom(scanner, query, bt, rand)) { + numFalsePos += 1; + } + } + double falsePosRate = numFalsePos * 1.0 / nTrials; + LOG.debug(String.format(testIdMsg + + " False positives: %d out of %d (%f)", + numFalsePos, nTrials, falsePosRate) + fakeLookupModeStr); + + // Check for obvious Bloom filter crashes. + assertTrue("False positive is too high: " + falsePosRate + " (greater " + + "than " + TOO_HIGH_ERROR_RATE + ")" + fakeLookupModeStr, + falsePosRate < TOO_HIGH_ERROR_RATE); + + // Now a more precise check to see if the false positive rate is not + // too high. The reason we use a relaxed restriction for the real-world + // case as opposed to the "fake lookup" case is that our hash functions + // are not completely independent. + + double maxZValue = fakeLookupEnabled ? 1.96 : 2.5; + validateFalsePosRate(falsePosRate, nTrials, maxZValue, cbf, + fakeLookupModeStr); + + // For checking the lower bound we need to eliminate the last chunk, + // because it is frequently smaller and the false positive rate in it + // is too low. This does not help if there is only one under-sized + // chunk, though. + int nChunks = cbf.getNumChunks(); + if (nChunks > 1) { + numFalsePos -= cbf.getNumPositivesForTesting(nChunks - 1); + nTrials -= cbf.getNumQueriesForTesting(nChunks - 1); + falsePosRate = numFalsePos * 1.0 / nTrials; + LOG.info(testIdMsg + " False positive rate without last chunk is " + + falsePosRate + fakeLookupModeStr); + } + + validateFalsePosRate(falsePosRate, nTrials, -2.58, cbf, + fakeLookupModeStr); + } finally { + ByteBloomFilter.setFakeLookupMode(false); + } + } + + r.close(); + } + + private boolean isInBloom(StoreFileScanner scanner, byte[] row, BloomType bt, + Random rand) { + return isInBloom(scanner, row, + TestHFileWriterV2.randomRowOrQualifier(rand)); + } + + private boolean isInBloom(StoreFileScanner scanner, byte[] row, + byte[] qualifier) { + Scan scan = new Scan(row, row); + TreeSet columns = new TreeSet(Bytes.BYTES_COMPARATOR); + columns.add(qualifier); + return scanner.shouldSeek(scan, columns); + } + + private Path writeStoreFile(int t, BloomType bt, List kvs) + throws IOException { + conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, + BLOOM_BLOCK_SIZES[t]); + conf.setBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, true); + + StoreFile.Writer w = StoreFile.createWriter(fs, + HBaseTestingUtility.getTestDir(), BLOCK_SIZES[t], null, null, conf, + bt, 0); + + assertTrue(w.hasBloom()); + assertTrue(w.getBloomWriter() instanceof CompoundBloomFilterWriter); + CompoundBloomFilterWriter cbbf = + (CompoundBloomFilterWriter) w.getBloomWriter(); + + int keyCount = 0; + KeyValue prev = null; + LOG.debug("Total keys/values to insert: " + kvs.size()); + for (KeyValue kv : kvs) { + w.append(kv); + + // Validate the key count in the Bloom filter. + boolean newKey = true; + if (prev != null) { + newKey = !(bt == BloomType.ROW ? KeyValue.COMPARATOR.matchingRows(kv, + prev) : KeyValue.COMPARATOR.matchingRowColumn(kv, prev)); + } + if (newKey) + ++keyCount; + assertEquals(keyCount, cbbf.getKeyCount()); + + prev = kv; + } + w.close(); + + return w.getPath(); + } + + @Test + public void testCompoundBloomSizing() { + int bloomBlockByteSize = 4096; + int bloomBlockBitSize = bloomBlockByteSize * 8; + double targetErrorRate = 0.01; + long maxKeysPerChunk = ByteBloomFilter.idealMaxKeys(bloomBlockBitSize, + targetErrorRate); + + long bloomSize1 = bloomBlockByteSize * 8; + long bloomSize2 = ByteBloomFilter.computeBitSize(maxKeysPerChunk, + targetErrorRate); + + double bloomSizeRatio = (bloomSize2 * 1.0 / bloomSize1); + assertTrue(Math.abs(bloomSizeRatio - 0.9999) < 0.0001); + } + + @Test + public void testCreateKey() { + CompoundBloomFilterBase cbfb = new CompoundBloomFilterBase(); + byte[] row = "myRow".getBytes(); + byte[] qualifier = "myQualifier".getBytes(); + byte[] rowKey = cbfb.createBloomKey(row, 0, row.length, + row, 0, 0); + byte[] rowColKey = cbfb.createBloomKey(row, 0, row.length, + qualifier, 0, qualifier.length); + KeyValue rowKV = KeyValue.createKeyValueFromKey(rowKey); + KeyValue rowColKV = KeyValue.createKeyValueFromKey(rowColKey); + assertEquals(rowKV.getTimestamp(), rowColKV.getTimestamp()); + assertEquals(Bytes.toStringBinary(rowKV.getRow()), + Bytes.toStringBinary(rowColKV.getRow())); + assertEquals(0, rowKV.getQualifier().length); + } + +} diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java index 9bbd428..70e4838 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java @@ -70,7 +70,8 @@ public class TestFSErrorsExposed { HBaseTestingUtility.getTestDir("internalScannerExposesErrors"), "regionname"), "familyname"); FaultyFileSystem fs = new FaultyFileSystem(util.getTestFileSystem()); - StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2*1024); + StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2*1024, + util.getConfiguration()); TestStoreFile.writeStoreFile( writer, Bytes.toBytes("cf"), Bytes.toBytes("qual")); @@ -111,7 +112,8 @@ public class TestFSErrorsExposed { HBaseTestingUtility.getTestDir("internalScannerExposesErrors"), "regionname"), "familyname"); FaultyFileSystem fs = new FaultyFileSystem(util.getTestFileSystem()); - StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2 * 1024); + StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2 * 1024, + util.getConfiguration()); TestStoreFile.writeStoreFile( writer, Bytes.toBytes("cf"), Bytes.toBytes("qual")); diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java index b490f8c..cec731d 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; +import org.apache.hadoop.util.Progressable; import org.mockito.Mockito; import com.google.common.base.Joiner; @@ -204,7 +206,7 @@ public class TestStore extends TestCase { Configuration c = HBaseConfiguration.create(); FileSystem fs = FileSystem.get(c); StoreFile.Writer w = StoreFile.createWriter(fs, storedir, - StoreFile.DEFAULT_BLOCKSIZE_SMALL); + StoreFile.DEFAULT_BLOCKSIZE_SMALL, c); w.appendMetadata(seqid + 1, false); w.close(); this.store.close(); @@ -571,6 +573,14 @@ public class TestStore extends TestCase { return new FaultyOutputStream(super.create(p), faultPos); } + @Override + public FSDataOutputStream create(Path f, FsPermission permission, + boolean overwrite, int bufferSize, short replication, long blockSize, + Progressable progress) throws IOException { + return new FaultyOutputStream(super.create(f, permission, + overwrite, bufferSize, replication, blockSize, progress), faultPos); + } + } static class FaultyOutputStream extends FSDataOutputStream { diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index c4c66e1..4387170 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -43,6 +43,7 @@ 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.LruBlockCache.CacheStats; +import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.mockito.Mockito; @@ -87,7 +88,8 @@ public class TestStoreFile extends HBaseTestCase { public void testBasicHalfMapFile() throws Exception { // Make up a directory hierarchy that has a regiondir and familyname. StoreFile.Writer writer = StoreFile.createWriter(this.fs, - new Path(new Path(this.testDir, "regionname"), "familyname"), 2 * 1024); + new Path(new Path(this.testDir, "regionname"), "familyname"), 2 * 1024, + conf); writeStoreFile(writer); checkHalfHFile(new StoreFile(this.fs, writer.getPath(), true, conf, StoreFile.BloomType.NONE, false)); @@ -127,7 +129,8 @@ public class TestStoreFile extends HBaseTestCase { 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. - StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024); + StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024, + conf); writeStoreFile(writer); StoreFile hsf = new StoreFile(this.fs, writer.getPath(), true, conf, StoreFile.BloomType.NONE, false); @@ -197,8 +200,11 @@ public class TestStoreFile extends HBaseTestCase { (topScanner.isSeeked() && topScanner.next())) { key = topScanner.getKey(); - assertTrue(topScanner.getReader().getComparator().compare(key.array(), - key.arrayOffset(), key.limit(), midkey, 0, midkey.length) >= 0); + if (topScanner.getReader().getComparator().compare(key.array(), + key.arrayOffset(), key.limit(), midkey, 0, midkey.length) < 0) { + fail("key=" + Bytes.toStringBinary(key) + " < midkey=" + + Bytes.toStringBinary(midkey)); + } if (first) { first = false; LOG.info("First in top: " + Bytes.toString(Bytes.toBytes(key))); @@ -327,7 +333,8 @@ public class TestStoreFile extends HBaseTestCase { private void bloomWriteRead(StoreFile.Writer writer, FileSystem fs) throws Exception { - float err = conf.getFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, 0); + float err = conf.getFloat( + BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0); Path f = writer.getPath(); long now = System.currentTimeMillis(); for (int i = 0; i < 2000; i += 2) { @@ -362,25 +369,24 @@ public class TestStoreFile extends HBaseTestCase { } reader.close(); fs.delete(f, true); - System.out.println("False negatives: " + falseNeg); - assertEquals(0, falseNeg); - System.out.println("False positives: " + falsePos); - if (!(falsePos <= 2* 2000 * err)) { - System.out.println("WTFBBQ! " + falsePos + ", " + (2* 2000 * err) ); - } - assertTrue(falsePos <= 2* 2000 * err); + assertEquals("False negatives: " + falseNeg, 0, falseNeg); + int maxFalsePos = (int) (2 * 2000 * err); + assertTrue("Too many false positives: " + falsePos + " (err=" + err + + ", expected no more than " + maxFalsePos + ")", + falsePos <= maxFalsePos); } public void testBloomFilter() throws Exception { FileSystem fs = FileSystem.getLocal(conf); - conf.setFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, (float)0.01); - conf.setBoolean(StoreFile.IO_STOREFILE_BLOOM_ENABLED, true); + conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, + (float) 0.01); + conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true); // write the file Path f = new Path(ROOT_DIR, getName()); StoreFile.Writer writer = new StoreFile.Writer(fs, f, StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM, - conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000, false); + conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000); bloomWriteRead(writer, fs); } @@ -388,8 +394,8 @@ public class TestStoreFile extends HBaseTestCase { public void testBloomTypes() throws Exception { float err = (float) 0.01; FileSystem fs = FileSystem.getLocal(conf); - conf.setFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, err); - conf.setBoolean(StoreFile.IO_STOREFILE_BLOOM_ENABLED, true); + conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err); + conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true); int rowCount = 50; int colCount = 10; @@ -411,7 +417,7 @@ public class TestStoreFile extends HBaseTestCase { StoreFile.Writer writer = new StoreFile.Writer(fs, f, StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM, - conf, KeyValue.COMPARATOR, bt[x], expKeys[x], false); + conf, KeyValue.COMPARATOR, bt[x], expKeys[x]); long now = System.currentTimeMillis(); for (int i = 0; i < rowCount*2; i += 2) { // rows @@ -471,19 +477,23 @@ public class TestStoreFile extends HBaseTestCase { float err = (float)0.005; FileSystem fs = FileSystem.getLocal(conf); Path f = new Path(ROOT_DIR, getName()); - conf.setFloat(StoreFile.IO_STOREFILE_BLOOM_ERROR_RATE, err); - conf.setBoolean(StoreFile.IO_STOREFILE_BLOOM_ENABLED, true); - conf.setInt(StoreFile.IO_STOREFILE_BLOOM_MAX_KEYS, 1000); + conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err); + conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true); + conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_MAX_KEYS, 1000); + + // This test only runs for HFile format version 1. + conf.setInt(HFile.FORMAT_VERSION_KEY, 1); // this should not create a bloom because the max keys is too small StoreFile.Writer writer = new StoreFile.Writer(fs, f, StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM, - conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000, false); + conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000); assertFalse(writer.hasBloom()); writer.close(); fs.delete(f, true); - conf.setInt(StoreFile.IO_STOREFILE_BLOOM_MAX_KEYS, Integer.MAX_VALUE); + conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_MAX_KEYS, + Integer.MAX_VALUE); // TODO: commented out because we run out of java heap space on trunk /* @@ -500,8 +510,7 @@ public class TestStoreFile extends HBaseTestCase { // because Java can't create a contiguous array > MAX_INT writer = new StoreFile.Writer(fs, f, StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM, - conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, Integer.MAX_VALUE, - false); + conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, Integer.MAX_VALUE); assertFalse(writer.hasBloom()); writer.close(); fs.delete(f, true); @@ -556,7 +565,7 @@ public class TestStoreFile extends HBaseTestCase { } /** - *Generate a list of KeyValues for testing based on given parameters + * Generate a list of KeyValues for testing based on given parameters * @param timestamps * @param numRows * @param qualifier @@ -592,7 +601,8 @@ public class TestStoreFile extends HBaseTestCase { Path storedir = new Path(new Path(this.testDir, "regionname"), "familyname"); Path dir = new Path(storedir, "1234567890"); - StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024); + StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024, + conf); List kvList = getKeyValueSet(timestamps,numRows, family, qualifier); @@ -645,7 +655,7 @@ public class TestStoreFile extends HBaseTestCase { long startEvicted = cs.getEvictedCount(); // Let's write a StoreFile with three blocks, with cache on write off - conf.setBoolean("hbase.rs.cacheblocksonwrite", false); + conf.setBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, false); Path pathCowOff = new Path(baseDir, "123456789"); StoreFile.Writer writer = writeStoreFile(conf, pathCowOff, 3); StoreFile hsf = new StoreFile(this.fs, writer.getPath(), true, conf, @@ -666,7 +676,7 @@ public class TestStoreFile extends HBaseTestCase { reader.close(); // Now write a StoreFile with three blocks, with cache on write on - conf.setBoolean("hbase.rs.cacheblocksonwrite", true); + conf.setBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, true); Path pathCowOn = new Path(baseDir, "123456788"); writer = writeStoreFile(conf, pathCowOn, 3); hsf = new StoreFile(this.fs, writer.getPath(), true, conf, @@ -702,6 +712,12 @@ public class TestStoreFile extends HBaseTestCase { while ((kv1 = scannerOne.next()) != null) { kv2 = scannerTwo.next(); assertTrue(kv1.equals(kv2)); + assertTrue(Bytes.compareTo( + kv1.getBuffer(), kv1.getKeyOffset(), kv1.getKeyLength(), + kv2.getBuffer(), kv2.getKeyOffset(), kv2.getKeyLength()) == 0); + assertTrue(Bytes.compareTo( + kv1.getBuffer(), kv1.getValueOffset(), kv1.getValueLength(), + kv2.getBuffer(), kv2.getValueOffset(), kv2.getValueLength()) == 0); } assertNull(scannerTwo.next()); assertEquals(startHit + 6, cs.getHitCount()); @@ -755,8 +771,7 @@ public class TestStoreFile extends HBaseTestCase { int blockSize = totalSize / numBlocks; StoreFile.Writer writer = new StoreFile.Writer(fs, path, blockSize, HFile.DEFAULT_COMPRESSION_ALGORITHM, - conf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000, - conf.getBoolean("hbase.rs.cacheblocksonwrite", false)); + conf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000); // We'll write N-1 KVs to ensure we don't write an extra block kvs.remove(kvs.size()-1); for (KeyValue kv : kvs) { diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index 6706ac8..a33c38b 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -199,7 +199,7 @@ public class TestWALReplay { HLog wal = createWAL(this.conf); HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf); Path f = new Path(basedir, "hfile"); - HFile.Writer writer = new HFile.Writer(this.fs, f); + HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(this.fs, f); byte [] family = htd.getFamilies().iterator().next().getName(); byte [] row = Bytes.toBytes(tableNameStr); writer.append(new KeyValue(row, family, family, row)); diff --git a/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java b/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java index d3b2f61..e542963 100644 --- a/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java +++ b/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java @@ -23,46 +23,45 @@ package org.apache.hadoop.hbase.util; import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.nio.ByteBuffer; -import java.util.BitSet; import junit.framework.TestCase; public class TestByteBloomFilter extends TestCase { - + public void testBasicBloom() throws Exception { ByteBloomFilter bf1 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0); ByteBloomFilter bf2 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0); bf1.allocBloom(); bf2.allocBloom(); - + // test 1: verify no fundamental false negatives or positives byte[] key1 = {1,2,3,4,5,6,7,8,9}; byte[] key2 = {1,2,3,4,5,6,7,8,7}; - + bf1.add(key1); bf2.add(key2); - + assertTrue(bf1.contains(key1)); assertFalse(bf1.contains(key2)); assertFalse(bf2.contains(key1)); assertTrue(bf2.contains(key2)); - + byte [] bkey = {1,2,3,4}; byte [] bval = "this is a much larger byte array".getBytes(); - + bf1.add(bkey); bf1.add(bval, 1, bval.length-1); - + assertTrue( bf1.contains(bkey) ); assertTrue( bf1.contains(bval, 1, bval.length-1) ); assertFalse( bf1.contains(bval) ); assertFalse( bf1.contains(bval) ); - - // test 2: serialization & deserialization. + + // test 2: serialization & deserialization. // (convert bloom to byte array & read byte array back in as input) ByteArrayOutputStream bOut = new ByteArrayOutputStream(); bf1.writeBloom(new DataOutputStream(bOut)); - ByteBuffer bb = ByteBuffer.wrap(bOut.toByteArray()); + ByteBuffer bb = ByteBuffer.wrap(bOut.toByteArray()); ByteBloomFilter newBf1 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0); assertTrue(newBf1.contains(key1, bb)); @@ -71,16 +70,17 @@ public class TestByteBloomFilter extends TestCase { assertTrue( newBf1.contains(bval, 1, bval.length-1, bb) ); assertFalse( newBf1.contains(bval, bb) ); assertFalse( newBf1.contains(bval, bb) ); - + System.out.println("Serialized as " + bOut.size() + " bytes"); assertTrue(bOut.size() - bf1.byteSize < 10); //... allow small padding } - + public void testBloomFold() throws Exception { // test: foldFactor < log(max/actual) - ByteBloomFilter b = new ByteBloomFilter(1003, (float)0.01, Hash.MURMUR_HASH, 2); + ByteBloomFilter b = new ByteBloomFilter(1003, (float) 0.01, + Hash.MURMUR_HASH, 2); b.allocBloom(); - int origSize = b.getByteSize(); + long origSize = b.getByteSize(); assertEquals(1204, origSize); for (int i = 0; i < 12; ++i) { b.add(Bytes.toBytes(i)); @@ -106,7 +106,7 @@ public class TestByteBloomFilter extends TestCase { ByteBloomFilter b = new ByteBloomFilter(10*1000*1000, (float)err, Hash.MURMUR_HASH, 3); b.allocBloom(); long startTime = System.currentTimeMillis(); - int origSize = b.getByteSize(); + long origSize = b.getByteSize(); for (int i = 0; i < 1*1000*1000; ++i) { b.add(Bytes.toBytes(i)); } @@ -119,12 +119,12 @@ public class TestByteBloomFilter extends TestCase { endTime = System.currentTimeMillis(); System.out.println("Total Fold time = " + (endTime - startTime) + "ms"); assertTrue(origSize >= b.getByteSize()<<3); - + // test startTime = System.currentTimeMillis(); int falsePositives = 0; for (int i = 0; i < 2*1000*1000; ++i) { - + if (b.contains(Bytes.toBytes(i))) { if(i >= 1*1000*1000) falsePositives++; } else { @@ -138,4 +138,27 @@ public class TestByteBloomFilter extends TestCase { // test: foldFactor > log(max/actual) } + + public void testSizing() { + int bitSize = 8 * 128 * 1024; // 128 KB + double errorRate = 0.025; // target false positive rate + + // How many keys can we store in a Bloom filter of this size maintaining + // the given false positive rate, not taking into account that the n + long maxKeys = ByteBloomFilter.idealMaxKeys(bitSize, errorRate); + assertEquals(136570, maxKeys); + + // A reverse operation: how many bits would we need to store this many keys + // and keep the same low false positive rate? + long bitSize2 = ByteBloomFilter.computeBitSize(maxKeys, errorRate); + + // The bit size comes out a little different due to rounding. + assertTrue(Math.abs(bitSize2 - bitSize) * 1.0 / bitSize < 1e-5); + } + + public void testFoldableByteSize() { + assertEquals(128, ByteBloomFilter.computeFoldableByteSize(1000, 5)); + assertEquals(640, ByteBloomFilter.computeFoldableByteSize(5001, 4)); + } + } diff --git a/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java b/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java index e70135f..dc6cfd5 100644 --- a/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java +++ b/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java @@ -19,6 +19,10 @@ */ package org.apache.hadoop.hbase.util; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.util.Arrays; @@ -142,6 +146,8 @@ public class TestBytes extends TestCase { byte [] key2 = {4,9}; byte [] key2_2 = {4}; byte [] key3 = {5,11}; + byte [] key4 = {0}; + byte [] key5 = {2}; assertEquals(1, Bytes.binarySearch(arr, key1, 0, 1, Bytes.BYTES_RAWCOMPARATOR)); @@ -157,8 +163,22 @@ public class TestBytes extends TestCase { Bytes.BYTES_RAWCOMPARATOR)); assertEquals(5, Bytes.binarySearch(arr, key3, 1, 1, Bytes.BYTES_RAWCOMPARATOR)); + assertEquals(-1, + Bytes.binarySearch(arr, key4, 0, 1, Bytes.BYTES_RAWCOMPARATOR)); + assertEquals(-2, + Bytes.binarySearch(arr, key5, 0, 1, Bytes.BYTES_RAWCOMPARATOR)); + + // Search for values to the left and to the right of each item in the array. + for (int i = 0; i < arr.length; ++i) { + assertEquals(-(i + 1), Bytes.binarySearch(arr, + new byte[] { (byte) (arr[i][0] - 1) }, 0, 1, + Bytes.BYTES_RAWCOMPARATOR)); + assertEquals(-(i + 2), Bytes.binarySearch(arr, + new byte[] { (byte) (arr[i][0] + 1) }, 0, 1, + Bytes.BYTES_RAWCOMPARATOR)); + } } - + public void testStartsWith() { assertTrue(Bytes.startsWith(Bytes.toBytes("hello"), Bytes.toBytes("h"))); assertTrue(Bytes.startsWith(Bytes.toBytes("hello"), Bytes.toBytes(""))); @@ -202,4 +222,30 @@ public class TestBytes extends TestCase { return (Bytes.toLong(testValue) + amount) == incrementResult; } + + public void testFixedSizeString() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos); + Bytes.writeStringFixedSize(dos, "Hello", 5); + Bytes.writeStringFixedSize(dos, "World", 18); + Bytes.writeStringFixedSize(dos, "", 9); + + try { + // Use a long dash which is three bytes in UTF-8. If encoding happens + // using ISO-8859-1, this will fail. + Bytes.writeStringFixedSize(dos, "Too\u2013Long", 9); + fail("Exception expected"); + } catch (IOException ex) { + assertEquals( + "Trying to write 10 bytes (Too\\xE2\\x80\\x93Long) into a field of " + + "length 9", ex.getMessage()); + } + + ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + DataInputStream dis = new DataInputStream(bais); + assertEquals("Hello", Bytes.readStringFixedSize(dis, 5)); + assertEquals("World", Bytes.readStringFixedSize(dis, 18)); + assertEquals("", Bytes.readStringFixedSize(dis, 9)); + } + } diff --git a/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java b/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java new file mode 100644 index 0000000..a55c212 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java @@ -0,0 +1,111 @@ +/* + * Copyright 2011 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.util; + +import java.util.Map; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import static org.junit.Assert.*; +import org.junit.Test; + +public class TestIdLock { + + private static final Log LOG = LogFactory.getLog(TestIdLock.class); + + private static final int NUM_IDS = 16; + private static final int NUM_THREADS = 128; + private static final int NUM_SECONDS = 20; + + private IdLock idLock = new IdLock(); + + private Map idOwner = new ConcurrentHashMap(); + + private class IdLockTestThread implements Callable { + + private String clientId; + + public IdLockTestThread(String clientId) { + this.clientId = clientId; + } + + @Override + public Boolean call() throws Exception { + Thread.currentThread().setName(clientId); + Random rand = new Random(); + long endTime = System.currentTimeMillis() + NUM_SECONDS * 1000; + while (System.currentTimeMillis() < endTime) { + long id = rand.nextInt(NUM_IDS); + + LOG.info(clientId + " is waiting for id " + id); + IdLock.Entry lockEntry = idLock.getLockEntry(id); + try { + int sleepMs = 1 + rand.nextInt(4); + String owner = idOwner.get(id); + if (owner != null) { + LOG.error("Id " + id + " already taken by " + owner + ", " + + clientId + " failed"); + return false; + } + + idOwner.put(id, clientId); + LOG.info(clientId + " took id " + id + ", sleeping for " + + sleepMs + "ms"); + Thread.sleep(sleepMs); + LOG.info(clientId + " is releasing id " + id); + idOwner.remove(id); + + } finally { + idLock.releaseLockEntry(lockEntry); + } + } + return true; + } + + } + + @Test + public void testMultipleClients() throws Exception { + ExecutorService exec = Executors.newFixedThreadPool(NUM_THREADS); + try { + ExecutorCompletionService ecs = + new ExecutorCompletionService(exec); + for (int i = 0; i < NUM_THREADS; ++i) + ecs.submit(new IdLockTestThread("client_" + i)); + for (int i = 0; i < NUM_THREADS; ++i) { + Future result = ecs.take(); + assertTrue(result.get()); + } + idLock.assertMapEmpty(); + } finally { + exec.shutdown(); + } + } + +} diff --git a/src/test/resources/org/apache/hadoop/hbase/io/hfile/8e8ab58dcf39412da19833fcd8f687ac b/src/test/resources/org/apache/hadoop/hbase/io/hfile/8e8ab58dcf39412da19833fcd8f687ac new file mode 100644 index 0000000000000000000000000000000000000000..cc260fb2e0512421f695d23cfda9bfa1905f05cf GIT binary patch literal 84964 zcmV(yKb;oOj$GW9+|^#oBYtRr^Bxo$5PLIazuT2t@qP;RgNtgK}cz${11{TtRaT z^$Bspc?LbC71UhSPFF$ZD?+7|5Q$=t{lOcZwvk_ee~zzjcbc z511BwLi;+TV&a*Y5W!Nl^A9A+KZosKY=|Jebs`jAIoUW3)|4z!vU+T;?XP;4&7@;+ z%_M5@kdKL!y&(ziT<|@C$42d?EVdp>6e9fnN#t&+VwFBR2Sg-%Pds z><#!8(D<8&m!M6P=CNCz4s0n6^RhA)HKRh{Jy#=&63ip{AGZ z(jax50ZF7RNRyYzb;MLB$SbxcH;rPO@hy)Mf++`^d*oz1mK8WFD< z>G7{qsQxn^8eve0*x1*;u0YK0_Bj>KHeq59o~XrLU@lPOE8FolqTIat!t%CKS;mG> z>J!U-n!Rg(QJ&S3v?|9ejnxO^OaV(|m&p=jE#6k`%pRKqC#-Qy;iPn5i00QObxbK9)%n7xB8dJEDxBmbqy1FFzSv#^ z9KV`9he)0QL&6-cR`NmG#hQgzWY&L?VI!B$)RFCunzBozVg z5W{Omxpc^)J605x!kkTuR#xpVYPj4&{g}ko(}ScAt1}B2sK5uxJ?bAtP@M$-pb`B? z61XZ{=pdQRc*IbVbFF|e^{8o0R*jV4JwST+KIyG>R0hcvNz z+oPJgNg`ZCnU!?WJpK=&yBNMoaRWGa*wGHssd!k`7Y0IqjJj-0`qi4)_LpRWM)Yp^ z=R-mxonIsI<@(;7K%q@!`<-{fxW{nMg7#wULo zwuUR-bX)B?mk6PjRQ0v}!sN>!JD_3(g%l~!1i`#G3Z?!J9KnD0h7fQ!CrWr|VqSa~k1d)RD?kid2dsiT`G4h30r+?^IkD{l3my&0#&M7(KKj<5PJYv4!4V8Q2{~9PS zgh0AWs5@>{chr*_jlS9y5rPN43p<^8@2UJTev6#lJ)S|gDJS<-SPJnp&UMYZ>B5!m zubp7vb_-+0eN4?U7NN>nT`Tw?uS#oys5AnucF0|jltrR*d?)J;f1+zy>U6h=Pq702 z&QJj3e^LZgFGN5r;-x{i(5=m$sZ2N_M}KAwuuGViLByp{_YVaeHT#Z=Wj$;clIZDz z1F>I~+e%&?{bDr=qfgSp+lG&F>PmklvU=hAt29kMyaC2xbRR86^Rk?1lwja+jiIA< zP~a7+(2WZNf!R$}uEMr-x=i#3o9`p6dSs)1tk$oNVEGpk_4ET~4!y`Fx$u4?gFa}I zN&qA|^SwJk6ei{k@>}b%6ywj^ZDhvUj&ek#iYBDR`1u!BP(Royf|I#g5FNGWe(Z=! zC<#5RGih1)zTV!85}}za^(z=6pf(XhV~6_;#**+=l-8{|yZkuT0N||G(y&kj1X8KyQSpQlzJ?lKHaTB-^6f(!;GMG z0mV<*FD0|1fGeI0rKq+QHk^@Y`z)StXEqD! z6GYTibT?*nQ->JfVuZPX(t43wXp4oLIo@pa{Yl*PnWgU`iFNxFm~b*WFpa-g+J^=y zx$^7M6CB^e)>VHFWGL7QtIz!WPCK6d>x%L5s<|VVH48nm;_FSmL?Kq^+87t?pe8rc z_t5Zp4aPq(un5X;XY^ZPPsspdoq8o4j3=fL57Z%JlZ~8;n3CkpnG-CYlwlpY6NND- zJkOhLhGG+*F6<)dzyM=B78G!0#(v;bvF;~Dqga6TXc{MyO#E=%&x&wjni2_k7*I+| zgvS`Ae@5jU`$82NDuPF%xvt$|Q@`UliFzuqjqpA3PkhIzYH|BSU%;MG2WkPhCgD68 zOrg5*IBEIB4qTl6rJ6u*(&!fnGz$k9-m8;*yPcQTZr$%q)?jaSSTqns%UVO8D0U{A zST?0X#l|z&l#j@xf?vxfurl|YR07vgToqf+mP|i#B>xHWEZrQnh;%4;e|pX`yQ7{( zP$(i7jEYaYbp(f^4m{?4dI@&A-7o)I`S~D6_NY~o*e0NQ`%g#e;|?w{;hLPZs_Rg-w8278pN-GOsxz! z_nz)>@ecGFE0PTp{O0PzI%hKZYW7PY5-oGkNFH?yb^E&<*77!;AViQOZuL3ct}s$( zZXnXzB^&4;oHq7_de!0)nXV@~l-afj+vBfTifYPOrr|?pQzx54(0hC}jF&mraISn3-_?t7N3O5xdDZoc@#yN_sWhJMI~tx)@Q3oD~&@1HV>G1 zF1@vspHH)RyDk%fX(>=TV7>Av=*cgJG023PurKfRgzb0bd8Wm&d_-QGRJ#|>oGm3T z@x)*Lo=g@zVUaFg{0*MRS;)+*V=XusqIqENv`wu$jTAF;76f2jhH)+Q*cKkyXmbZ$ z8!+pK+QM^3$`d&f%}?=(hf;_#c0hN{SNRpirh*C(2bgpT;i4?f7Oe(Y^P*NTBH~y>noSf7vq+QfA+3Y9Zy%N zrR&41+iQ!kNZl2Utpay2d0GQ?reE`>%Z<^P;shBJ9u)puQlAN8U};q({Ob5x<6o+6 zJF9mFK*RSd2@(`|Ra>vy>C|w!xv|IT=u)!q$u23C^{tUAWDv}&WW2yqXj?z!7TYC= zx_oX_F1I*?JApD{-k)820IqK~u^(kebC&k_R{iLzRQdJr;4)n4H7O%eDC)^`5suI6ZZ(Wf9gn%MwxPMG5+0y z@)u;Mj+qYQ0FBdq3U?;0UQ(_G%N$ICgcXOAE~8+m+2E>4)TW`uQ!V zhhy-4UBxqafsb(xAI_apRiW$5AQp~8%zk>F=I@{r5^K6^V&_j`DpVg{cw=Yudhxh~ zP?F99yDgNLaKW!S`l8P?8#8GXENBd;{ud}UOO!ZwkXC^oVU;YXt#{^V%n^MAhm>H$ z`e^#SJLrZdA#_SmnEs5myn_xQdQaFlyZ*)Jh!uBQ_<$7)G6s{PZ7sa+-?T#cAE{}9 znFby0ZCPxz9QYn>4Z#yC{!CSIHCQXm$O~=j2_O?~srFiEG(;5AH}7t$RE)M2#WRdE z@9H9(a8%+=7~fl{>)_QA;TuV2^cs_t9<0}Zd8y6mLfXCX(O0;Kg&bCbbJ4#hr zzs-40DM(@hX5OR0C?~D+4Tpga417`}B+NK1cH7va&Cue5>V4RhTs}x4@V|%c>5){F zi9=sG{Q^t>wdAci0_lKWU?Ce~(TRw;qW0BF`8*I!vAW2KLqQB>X-jE;ecqRqlM;|k z8Np{?;~^%;u>wgU}rssL5T+#F2}!$^K0P zpChLo_(qV$nb9zJEY|u*mEK#W&BZd?>^YYpAM{Cc&U7-VU@+VAhF%YcSCDnf9k&r~&{b%*)fb|VjrxP- z6%k^uM0xQCnUyGf-fD|77<0vWmwB83Qm(r(8I5k7BviFh^*>&e|8aAXd!z{)@R198^ zX%Gh&N0VBP$R<|qor?7GT>FS>E!~`&-$|fP#%tm0k9=0f8v6)u8-FSdvJ1E5S?o5S z%OSUgykfp^nnUPNC;g3HPGmP#Dw;-;=O^N#6Jjc7oY7_9H!;%r3;WPBjdi&=p_V5V`_iFLPAgZk?n6;x$9ns z7To*$(=Wm~;MYwf>}#mM_?N5PVWy84`!}e2gQ@px|^5;SlLR+3AFde$u;)PUe)x4 zbKE@djrL`4q>$SW38e00o%433YbjVe4{zM zb7Oj>(%-)!c5Lo4ey#EVwB4b^&3*kgT*7XsebVs5b3}@SlLk@I*2F2oP|?___U1dv zig|&Av%5|BVJN8B=m)Q&`V!^5*>%phG;Qh8yZzfvbd8l=5ko&Af%kAIN!86ZXYh&m z*FcquopG@uNk?^552{Ya6bbEn#*cv5uM{ry!V%g%w=T|<1NT+4>RpWx!2@uk`NaHa z`2ISqFPBr;p45B^lx*j6+@ja}bScpbuuuCM_jYfgu?A?YjQkd%8Eq!Wz{Ja*x5u7*exnMX&zruSXTK}q;0AF$e#SCCj>KecyT1taE;; za$IGKw}2yJr?hBag^w$l-jlv1sV2T9}(|`O{SV!=x{x5ufXC6LD zwK(&aQB3=nJjKAT&!zp-`gn6UEo~c)PjJY$$Q$O0fsA6+{6rQ&duH;y4f*#S_%qf| zhZZwpEbO}qI+SzM9m~o3tYaQlEt;TE{sYtc04rdhe0hVdv0WHd(#}jx1-nT+=MmAQg___8KGr2e%`;#bx_=f- zc{ae8M2PaP2P4CX`;K5VDy;tdb$^-1`_ZYKC`WM8iOZwSZkzKSWJi}EE6Vn(qjCXn za+QAZ`vS8T49xI1hN=wMMWoU{)wStiCu??w$&YUfhT8S@Io?*41o%B`7w7tx6*}uX zd~FvVe09N#mP(9-rF}QwvBn=ti2+vX8&;5=%HwkPs{&H z$b;*vxVVZg;A_n2v!#+FXvfBG)J8 z8_2cDSA&jZj9i^TL!iKV;Ebwm%{lSESD-=nM$Z@iwWT zfF~#E2N^vz`-D)#Athc>L94W{w%eFlqE#IP_%$@>50Q6(Ok@L3iC7;CA(@OJQ+NDO zkO+p+rx@9xq8&5lx9rrPU?kGzQ*%L<^*rKiJuUAwbwFT4FPRTLG`gR@WJDPl_o#+{RBH5&20|Wqb{>>^h?#~ zJn>l^i|ywtU)l8{m9jDWwhv0(mu+_4kC$4NtoACx4}MCk={o>v)Msy$px+5>n~8Un zk7HkO3lsU&Ev!cq&d(RcmN_3fD(FnS@sBz0>1k>|@1?$a;-yWKw))snkH;HP z`_U8NT#@BWJ@ZX$Ea8hOPFf2F^aOB`VD)T5P4X43kl*+;iU`3Q@k+uY$&Y-bN-#rI zNcT^IO$l_JF9|FcMwK08PD^9acZ&tDJmo{XWz{KY4j0Dm=W5K%2TC<$gze|q{n-Dp&b8cZL<>zG0EFK-uUDzL_F*4jaOxYr zbD=CR^eQ_qVySp%j%m4IvdiRu*;ebOa)mk@SYid8z}cv32rw>j68!ern@qkJ`V!u< zk#>WZtj!FgRvv0xKx7lvHGVdp4Nn>P5eY74Se%-Y>0PE?c}wHX_t$87(I2G*$K`qW z!ui}PJ#0uCihDt+u2TqdMaG5A*zhL6pynmi-RdIrG(;} z%HjJeb6YgQxYT1%Y;B3o=I0q#?8aALza&28#e$E7-75_5!qbNRICiF~(&*s!k0aFW z(gc_g3$dO}u39^50>03n8Xzl6i!}Yw=JwA?XMZ~f;nE8gSUvqsl`$e7O27~)uRkb| zi`@;C=@tAXJ*k+j_l=b!{`7ayT-B12Ra10VhXmb;xk7Y1*8rmQ57ACzQ1Ph!GBj|x z0-$T@Pixr35#C$j&c8mBQ^Y@ukat_yG%t^r-BBD?{7g{i%X;_k5QXADu~&UN=@_G< z3E`k76}5=>g_7|q)=suu^@z9|`Y)Reu%xqBOo0Cb%RyR%GNz+ryng;|$|}|3V_L3| zx}_M2PSP(}O;M&bZfT#%KKlQ9`WOCvEQr{6a0w|c6tUuDxXIB}F6BL^M{Ts@Ea7uS z(s~bvR1O&Esr*cXBT>OgiHG>8;5JO@kVVw@YfRz}lU9Y*{931s+m3(D#IB-OChd$G z4HmOaYXbzh6>g+Ruz{-DhlucHTu9VF0 z$-{OQG#{NJzw_#=hfcI#0+an#$eMl3+G>kF6VkW#UNFv@Az8&dwBq;&++CL}EXAMn zvE+;y;s%J1DO-nUlUvM#U1~vKWv)hW36>h|to9rKOSo0;zBL;RP6_{_uK?}JnX7Ky zh~-Xd%eS5i9oViSQ+otm%BVmK+8MSF-0h6^BN9;wq!8M&X+DB? z^?u~~i-I>9{T=f$lqfm^M^de8x#+|>G4}r_C!f5_z8vEki+_J72kkq&RYhk?3zHVi zo)rd_^%J0eSdqLyX8zV7Q1G^VQR`{D+`eu6zXb z%{@h#Fz}E&cga3sHA^^pyh;3=r7FlHZMgxL&c;yj6uPSM#$!1@WzNZY5(Ksf(AH2T z`CZs_zo@tTC2IW-9bez~!7He0iR@MH%(MZ3D|_V<%TIC2Qvf)Kg2K{Pbeq}W(e;^8 zhu7;$W68>J>$K6dBSEJ!eHwdg@ z(G}!K9cD4Sy;099eHx-brNarJZJQux)Ti18pg4|UF$}A<%|BjosKAfSm!zcz)SQOh z2ma0bo)u$ong!8yiKuzXZ^nLvCW}1~V*LvQwjk2eBf8^27-%H?_M|IaE|9LHQiWBT zt6=$;geDo_gt*kcXDI(`#UvQr-s|d3w@6rg4(rPK;F#H%RqC{9~0W z-+_MUniUJKvie3V(iOJWo(tD<7SgwBQlQZ|J@_%@vf9BdKPs2RX6(5zT>weN&Eif|7b!E$%!h9*jn}3}Zkw!& zc5CKVu?V5;Q-W*!$I~cqvkfgzierRq44R0^urDcEQt3%aqr4f&`Ix5Zr+om6wTw>* z7u{BphL6)~Mu%TNaOv~bwb+x=>1sgz;i^4^OV3fZgAeF9HLw9Sr634A`F)hHjgM+~$2y3GKgR9Du;)$VQj#Dj*KXxvJr&4QU zKLJ5Okw`2x9rXvDRtHUEv-_a0o#vUJqU`z1Q-@0;0Aps3KWRr9(p3u#?-Ozf*H3OQ zzF0sZfxtGa#vw;5^(%`(7U|_p8n;tk9zI&Et|4q#h=Vr(3@2miB|rS0da={D$uufB z#?8Am$oC5w@kTz+Ff@Oz92)D0^fCWh?T^uMvUs%0%Fqv1f@7Od^0W0Tn4`k)%96|r z26pyOriNz})L_jgd*yNm}5Skp%n#O~*6@V*Aq`fDqZ}5D(+*@KZnR^_9lX)!q7I#2`=2vl20jG{0 z+1^I2G@AjoYMup6f0)lq z4AC~T|AJ16BX)t}HtXU!oyPAHTO>CJdev}sCna>}HlyaevEPjUt8xD~^0xZ5eA##~ zj1;M-r8wt6IV{ zDtA;_Q`A!{jD-}}RA(N;%gG|LLV^3kExf7e)%F)>Sc8OrN>DUQZ*_(0@vX90pE5h` zL`#qhpcMtO0E4Q3vbssq=4k)6)S>e4wO2`p|48fZZRj$Tv%8xooG+jA%P_C#L$iaU zle-0hZJuoIpuR2>ADOI>MZs5MG`gf3>5DKrnp#vMvYMhu&P<@EL2;o#Ldemea;nCh zv6XU}bP{ncLv3r2<1sWBx8YdMz!=+4Rh9+^o#k(Vz$PfR4&Njs`&q=JJ*7yc_8Uch z3NB$Ul(aG6N~1zrcYH&p@S!kWQ)|MSKp{xgKoRWHHp|yoDYB{aF6$tpW%1d!%^pl9T#c zgp@9LRnlfg#N%1?>JjDv_;d+{M}xip&t!e>J#6B1;S6Q_+(_J>HI-p3;8VJim94N#Y<{jP(gi; zQ8GnNAD>y=AM@N+gg-up9E-gjA-hf>@+E!s@Qcw-sYFy>awRE~K2zHD-%Z@LbfPJh3Sh76}vh9xr+XZl36rKZJJt^w!8)PKL%sQI) z*8ainhJZ9@DW5l}JXe%`%^b@*IPWn&eg|4iw)>~2GsU!ptb1PO~;L+1MHdPDN=sjm9Q`b{%WlWghLt z8+S{>qvTH(IQc=oC11tEt>(M&PzL0r3LzhR|ClR{&@u>Nt3)-FHQap^)c3wEF)5b6`f-*7BaO;GlppszFR&9^ zo+@nkG{aGn#}l|DreO++U-jsP%%4D@C%eXsgu;d7{E6~XmMDK&eN;<5r|BM1l`X)H{NdXw?-N4cUN z6PZjCmyPw!oJnlnfy<4qV6Gqbn=z~Nv{4;P(r#W%@Azcx_>g08j(9F7qt`=%!c+H@ zG9t1OgUJVGf20Y$50{41;o3+zY{r^$Dy#l=JV~IHVlXdLp_@ThshUUbaJFs1Gnyly zoK}&Nbq>3{Mn0|@9svRyhk3yXa}v9DTNjliV+b~R%gvIfTlDR~iT_qW78B@f;;1t( zWvQb=;Xcm3Zdq$4)-n_%BbS_my~smj-fl(F^@sC0oYB~od*#N9sot(xr=mK+eB{Jm ziDz#7WXCoy)!S9FRVGN#d4b9vrAY|+Nt7Gs8N}|syvds&?-KaP?#8nOZB5#_I(6``fN|d|nYwph!ef+Mlw{NKs0Rv5B zmZBg5n=Q1EyB{_ienWpW$e5BDLf~7R3Wxp5EnKY9u=8k9U_s#pj^}00uTFRgZC@_* zqxrpO5>So=8(!nOdq!v&JG&U-!xdMIq%CRT0>@g+cW<t; z16Z1tEtGkV?A+duTy!Zed&3s$sOB+fg{S7MpJsCh5(%xRZF>!Y&UxG>>GzF&sdx5z zwcx?+pITP(DRBo4mQCj}@8KRtCE^UH@}ka}9WZ^eH2WThL~m{r)cIkv9AmVgLIbI$ zDd8Kde|LuM5JSI_K^Vlgfa>-e!n@TRGZ2odr$H;+#{Oz0}^BsdNkp<5Z9xO9cI zC<{cX2#mp_Brt>`Ub6VzCk&}-g^D^>yEQZ8hs}^5ObBY<+)fUWDFL4kMh|9DzfVqP zBt1E*MHn^TWo7#ONnv+LoY{5YHcR+ag~jrL5kU<7+_(!-Cco}M%2NWoLJBHqTkAe7 zrM_4KrB+5%z;?9Bn5)z30PySEM>Zg;&@9}m0L88b%Z9F#t<%zUi(3AIox_m%2F^MP z&cg^!lJWyghL5jaegOv-fGc;J5dHqmRIY2;n>by7~ozPhvlG^<@$ep#FU~`G2e3FJ)+g0^c9%S)lIgS;`rmljgqoy;56!p-I=HQM2<}bi5Xz5W zz3dfUd%Bm0} zM8e$w&7a*1iWO}i*6W^1%h_MeeF^2T-hR9Sz}>yUkSvwvrfpL^(hNpM7||IzTB1A3 zBy6kd7}GPT0Rl&VOwQm7xg0Z%EV(B?qSbu>&Ab7cm@MU{YRGI+iA_3RqmCNqC_ zn8I8(jWN%2rUL3C21pFJ?BYPJxPFpckSj-U)pEiFOzfCU;r(}iRMU<3!vr!jsr+hF zXtp;n6FFefumt_DpLvnpNM+g+D(jqOOh)Gs)Wdo@Z7AcQCwr$@;PYmK?TCO4^VuxnQ z`c;NQ!p~@xg%ot)loWGHHR@Kn->rt8ztizU_s)AC)Zxd$b!jlDT#2tB|6g(PVsZ@n zJv?tY^A_YNSEqj%E~RphUJCVmONATHG%~x(MRXqgPKy{Ym*PQv%LlHe#eR(w_`(8*Q#QAq{R) zwQb8j$kUmXss8U7#Y^cBPgE8Hr=jakxSM?7m{GOiT^{)ZkHgef4>=~FeI3`zHqQmi z5>{t=GTrT$K_^o;L7M>ph${CaxxalB;*%I%2d3gB>BoG%E;fx}0$Ku3)+8Z}6gs z3#ru?NDWg&_Fg}oBqgJ1m%H~ClE94|bsWn8a?+n*oM7k5Y2)a!{`K_r9!MjL$fsB$ zX(k0iKF*rjDU4;;FUta%OkVj_nJy_fAh54to*k#W^}~-|E;MQ(n7b4Sw#@-*3Jp3qG!dWaCRJ|`+9U}g+LbLf3hn&u z2fPCi*vvg?3i~ER`Z)EYC10&@T94Oz2{~tpk~ha5M4P~Oe?Ve@f0E2zwKulyYkE{0 z=N**gpT5NdHrrLneffN8SqIn%((@x1*eAJo%dm(#5ZIH`WE@q2-G@)KDz4L-m9A@T zz``*DdRqd!&>fss2nZH2^kYubIE~N5V6=u6Q?+`_n0Sz^u9J*>Oi$tD#J*XA)5Leu z%cv8~IzjMqD&=HD(}H1c{UX0@jAxJD%u>nr9NYD%7@ki87*{eaB>2UQg@vY1kK(>K zob&Bn%iDL?RP@7VGBE8+StV2*-PZF=6^o6;;Zaxr)s!znoa$A{gE*fWzxsXoL$^68 z|6J+_JEjs)6*0lNB*1v1MpG@QA90oCF_(uV*O=8MbnZHcXl90PjESx6Sa{)DY)HDUyhpEB$UlbESV2=gTNgTb3 z{5zX5%DcJ*Y>V}vn9fedP!Q07h+`(&C{gpY^Jcx#sRhR$Zv?L&F6FTG*Kcwra|W{{ zQa?jBZ<|x0S~3Z(y4N?R6?94s9)Q61JQ+KmFl-^85&$ z2fzqt?C&i$iIpc2=Q@sEoKI!9v;n>t|MX8Z>7cJ-b!IDW%8Jc;$EfHG`57$xWpNSD zXv_hx-hR>j%#(0 zoVM`3C`QU0lO^mP2|wZMNrFFx8cnHoql4uAxi@tT(yMH|^j$m<*e)uUR=jVhrCx0E0~h>w-CuGnk(%%%m*s=JtdMWPKxYBcibx5( z&`S22pt{;1w5JVr`VFQA!#X_QY9t}e!^O@_S5Z;J0Lp}Q5&5}_^3K7e0tjpfnRmhI zAdQt<KlfnvgotfFCi9FR6K7;`8QUrcBjLJq82o3%X^yQ2w|;vR z=An?c`%S!jWhzRAHj&DoXc*~LH&yV^&LTN7joZh%dZJz6;^gWhz3q=3wT+Q!U zq*0^#@RswXRcMs=m>N?JAAIORv#Tn_dC43s4f`C|f!zYTo0X7*;> zfcixPt;lZJ;5}Tm=zCf$l`y4En=7;qCKXKMKhQagT^bdklDieP9hZpn(W`_xiwA(eKW?UCx>Q@~qca z8GGyYf&wMF;Fi3s4o}8xh5^K-MhrZjJ|5{p7%anzDc0P4LQDiAHfy`(Pgv*vN3C8T zxUvZkq+ShvVMIw%K2vSXMZp$2Tv2UG2SM8TJ+h2T)M-g5-2Sk6+~1Fe#=x}pogq!( zYJEff`X-M?uyOQ`27jOBDO99qH&pM|?Vf8#5)#%jTt@H0zc0P1{^LaHtRUW7Bso%z zlB2^p%r>sV>{_pdRV^XDMkK}@OluurNIldWfd`+7liQiy_>7!VB8HO`qXk;=v{?r3 z_byos$A~8e%+>p#%`<;y&O{Or*jKM*y-FLzLgq1PQRsch-&Ch{bd*Ko6}pDK)y?@X z;T?eEn^Z-915vCCxX==&tq!b~iN=9X4Coi4c-X=r`Pf4H%_z+Gv@J{ncS`9|3tjNg zQXsH(NO_;?rJ(i?F|IM*Hzb+W=cdHP(9k(2hj1+JEc5FOn$krd`#!gG!R!k)(;Jv4 z++&pSUw3CnG;|C}*732x%1ZR;(8x#{)wI?T;BVpFgTN+P6+#;xz3edZFv4^i#M`Y6 zXVKDsN{}8~2i5imfs->vMUf|#?Rrs1Ere%Q*Bj`FMw{x#NNxMs$8JBzNlPqp1wv?< zhM|qulM(oSF4m4B@5+jRa$JM02nDv zhM|uNlFB|ixglyU-qjZ!m_UGdzs9!UpWZV#_U(6GVHK0@HihXyP%h_ip1j2x2sD8K zRR#GQd6qk=Bdt#@fSAjL`(g7GJjb#n{)%>m?RzpL%)G3T_t3 z8vIA+(>vE;(u`ZKH2Bi!vhgFp_~~{Z`=@tN1KCS z40up1O0A19>gw4_8`$QRydQ;oN9KM!AdlI)_MEH7!_0|Zxf)nkOJrMzsL+l+gSHC~ z0vmYNtIBRvze^BG_MrG!?iA1SgD(RbCb=A<+~ysi_2tgb-Ewkceo;PBKb(}($)Gq# zxKKNK8b4;cFT&1I?l3NrGm42e;!62Qkf zX68Imfp#r`fkODXJ#QZrXN}_jv#6T&lr0Ks(4%(L%cmDkt_bt z+4(i?vz{i!NtJZFwEY$`bZn9c= zXqln5yN@2iY?#G+^~6hbv^wG`E~l;$a(kUa?KS2QEFn7BpaVtE1KCWZAMG}A7*m)@j%5g*&iqEq z4x(?Zk#OeA%w|Q+jqZF9B;kEZ-M_aTo_&#`Qokw(fxYByPbHWByR9C0(!&@Rl#J`05jBhq8a)$kvOWmLUI#j_xr8(tphBp^xJZhDnDOLgxi{=k(Eu{ zDaI|n`HJ5luqR|4DI0uL>{J>E)xRrsnhaLgotsYBxxRmDp^AkgtyCY5-!=SuqrTZ_ zSM$||Haj;B0zySMygF$~k>Vt`&6QBg=SDCJ2CF|2G~ajg=T5M!7T*K}HZkZ-!1c^4 z)YqSS)Pyyvh3w1Lo-8hbMpsh4jxYeYoD&2BYB2LuO>xhCS2<}ZCVbzvsf2}=OsP~s zBCUDIIim^q2L;~XNVXlPA2de~hbSR}z&`Bj^4mM&ao_DhS&AJfkByU*6|q#-#aaxX zE6ZF0;yapTeA*C^I$iERy=}{jOU5dL$cvKJYpkGUIY{xpbu=i|Glt&Aa$*~A?|sUv z2Z6nO={DqZ%%U~oVmCKUbr@dnaAdUMHL_?mR2DXylyd|E)1zibQze}z!)~cXGc*l7 z*l*zk_U+kX0*IY)S${hdMz1#!P_uFbfVh^oAu37vL10sZp2q=7=MrR71)T$E{pt9+ zKfGS;&V^repYctRTbQcOnmr2lgSX&e*WMt182f`&@SdC$+hfNWrXONYG`Pc;ff{?M z4pusx1NZdArwau3MOZKLF_~F=68&aQCX>7jOze^-e|3`F=dwickv5diBtYLsto5YO zxibAJ^vNsP?_A;hzHU@#GPqP1=LrNUTS=Qn{bIhDQ?gr7Uqb0HXemHoBhbEmBfJ2G zMZVH!{jSA>IZ+1k?+i>q`pD&)>%D1Q3B<3xVV-Iggf0)e+ zuRUF}RBbgM^Zc>}fjy|{w`tRA|4>3kgYx54;`KlnFWn9pv77Wb$b9WhL5* zypwN&c-v+~xQd6?pxVchU=dq;%m97ff-dFHiQc)}!notAsl9G-J54zM4M1NUF;ElS zfwvTTmx?}tA;Ek-ehAfH;u8;4dRuV_q-bP$2rZFX4bWmxVXZ{9h(4A25Mq4Za4jdV zR_-(ZgB;c5D;-T&6;`bLT06e{{evk2XP-1#Kn=HMTuuD|F+k40t+1gUzZkDXr2FgZ z0crfIil(1|Av7JVS~)j5c8eF#h;?O;aDBOHxG0kQA;aqSDv)E4bG~@L5S2pMZuw1QdU?D=nvb4n9!mCS#iY9JxD%)c_>MT~}GQh|0S4niSY`R&9b1(((MQ$g<# zaiaiIedZtNd_dPD`)nUAb-{*-e?H2k^59^>Z##=1J|PE_4V2S*LZiWK&Z zE_b@}DoHD<0u5pCLSvYvco#)4qW261OZaga(s^ao9PCPLwcYQb&^Oirhg282 znjtXjAK=Uv9UQl|+|M=_@L5-G9%Mmaj|E1nMT=4J_ixe(mm(vrn3vt2WJ-s6GP*s8M!;5jTL3(B;1Y-b>*-_68U+U-*Ai zy#-g5-S;(o4jho~lrHHG>6Y$By1PL-B&EAUxF#cL|38QK!+F2JHTKwR z?GR}D3N!yK(eZG zLrfJb>1JQgv`SdiUwvuu;p*423(pPiU;tn+%BZvv#u34%XVD#{()+X4+jQF4KER5x zx~x$8Hpti8S=KRejf$T>s>V_FeMJL8ea7Rp`hW`+?oJW5WMeuv7Hv!RK0xj#iaK5i z{A?J6@^l>l3}G9?PGsh|)v)c^a)Cm*p-sD1dnS$c8N-rPL;#73P)@Cu=u61F>w)Eb zg}`Zi55J@?p<%qO>SIrc7S>3Fot{dU)U5Y0?76Yq3;-rFG#<9<7&Qq9f7Rhry7P|@fqapM&!mh26S?5E4m-5;4z#qobbH8!)fP9b zkUbIQ+f-}r15e}=-?@P&^F3PI95lyXt05jnLrM}jatl_GvR(RU!#3wIR^!N!GATFf zE|4lGk1}iHXJcz0+F$?lxwXZ9`*3WT?DukQ-{*^KB>yEB`T`Nd`cZ2yn*W~}|DJqO zOR(IFl3-w~Y3X!NT<)4fpp$(b+x9!P7eub=W)4h-mhbq{i#Y~0%~Yf#^fDguPweC) zZo8~~0>@nS2snjzwS9ILp9^F%<>dDbdj5Ie-xoABbG3CIv2&uYFR|ifWbG9mq{RdhW4BzCgeSBvFZZX@-NonODS3 zJHeNh!f2RfT=K`%@iV4J*rzFo)e|FfXUwo2V=)TbVBxl~O$Z?qX1L9{M246kK>@&6 zg){h3Wrhh;LL1#k$%6L`Wgn5G(}-4xF{-}9{*VH}`0-?}9zN3GapWh^Rbh6{(-hl$r1YFUYz6cuCGcNf_#3rm`z5pMh++xM?ohPzX+Fub z7*e;`=dpUlUki2DLFGq27rA%zqa z(-W|JOafXQ%}9x-jZ?WHVoxV>%&KpsS$4j<_%NQ+yrepo8NTQ13avDCMs2!##f)kMPe6+X z#g@8VqyJMYTw9(+i_c@eXSirhSCXxE z7jk~7i@J;hfT4TK#oU zbdx^1Pw+c2mc}ne?=960`xH}r{<`&wyw{dK{T7i+M~o2_ zjkKW(ieg+e=UzV3w|*pVZwBASpS@)+QT@~R>Z;JjKL~vv$uSY;R$!_e2Y#NVrnT6t z@Q<`G@{;&1ekZ2o_GSmX~vQm@wxRV;%Lvb2;uLv&|zle*+J-6P#po9s2~Ea|msF2fA!- zx^Ht!b>CUd?KOrgNDT6HK{S9H0N8wX?dcbUcO0}x9AjC5&`~mg0m(5Oq^Spr6`rM2 zZw~E&J5myFRS_;OK#MKbu{2mC9b2V0)S^cJq2ob|Y=e1&D|0#{T?d{D)+ML^OuG|_IWwHu6X$OrYg-9czbd?GBzEbD~5x+R|+=J_#@+igEkXt!** zM=nIEOAfDnI%~o^xRNKL0sw0-HgM~{k{%eyGK=;F(rM{v?I(|rl?iY%;b&rJftnFw zAxu!zPtDeUED!FNL>j;sURX0r=w1)gO}S$$M*=8)*1zi3tjAnvB*4cUumHez#Wy3l zaPZht4D9c&8!3Ex583jhw#k2T-I60=3(Ulb!;0FSmj=0G8M*#2*ZdLW1L@MsIsc+f zWv`}DskXGp^UUlOBgVsYmTS++0bbl}-N(g*6AZ;mM+uXU3HgVHVNjcTjSz-3kJeGh zpL>BOod#t-(JjZ}^!dc{YPRlq6@lJ!*IZTiUoCy#bLhXrbTwy@1KJPqFqx1fY&2|R zPhCa)OXPpU#`e0xJ-gJ_$^&KNFRFaLuQT+8ojATopKq&>Ts)ZVG>Wj4-c@doMMSY3 zl8F(Z-1~I_Kc|c;F_v-VkXq9~Xi9&%jw=XNL9ASWhq(L@3ULeogH?7Wl}>h;ZoC%T z0~xq6;eT!9cPP@GxY}o?j{d*}3CbrL4(C^tPNBcx5CrGvj0!raVu`frNt;z;RIQF} z5-+ih>1g{dr)G&;1hWcHf_w8WD&hg^Bruw+$tpLJ84j78{;V2vT#tzo$-tmrPz?zR z;(NsC7ixKG$ogNn5ilCw#HVbNrD`AZN(qvSq4b06v23fBJ93S^o7*De6Nq2`TcZ0L zi1ti@dCO62-X!E#a;!a*W7t@J%k66Qx(NscZH|Mm?4C19h)zh#{C2-oB3V%4hM&FB z63k`{$rQVaFjf(RaHCdnH4uC80bqL_`$n++m=&3LAN6;#l}E3CU-ZnwI!YR;x2Y%@ z9D~FeL)|r$MX$IBYm3i6m{Sl#QF6*|zACG$rkgdQWv#s2=!F~zdS1KR2mE2{NvZEq z+seAdQUri;c@EmkC~gdzamIr@r7vWc*T+W|nmdMLL8{ImZ{0u{br-fmg1|8>GW<`Vc={W z)r7JCWvag}eH8EvlI$KNOA2@E=4sPCo@(pz{f*rcol@#$mTnfuIZ^4w4*vE_oo|{gM?P0zniJE0*2-_adYxY z%#vm;Iizkpf(G^@8ZrQk(0bBPFoO!3%{14w8sb{Va~#zTuj2YIqh?En!5f?QA2Zg5xlp23MlFZ`v#7en~q_c!!XTa@4l5$5Je z$3~24^Q>sz%iv~fc?khBy^)EPX8;(Kch-(Pwnps7!w&O3usRo`3tgzgV=H5WA1}e> zGpHsArf9Mn=rn$x_aoNuVRKbQ{j5dcUQVeNUfgJc1-r!($KPAJU{C@E)sY&f3IZAr z07j=fce_}iI3XI%$=IixPP;9O+W*E!3>if`0n@<6@qRlz7;<_gfE`Ga;$P*YW=KnQ z6?6dGk1y&PwP@Xzu;U6hlz&khnFyhen)A&1O7H&9JN|+Q7oUj9IVuZfsYk{N&=Tny zmHsS0_@}(~NQ(|_sEQ&PeZw%<`Pw6J$VJKPMzWsRz0%f>Y2gUbWQspx;24gYUl^AbUh>`~?@GtlKs;Dny^v#|m33Dy?Y7Z`Z`lgpk3 z7;VQ74r;ZB6#VE1gD35UMRS~l1zT7jd$^G)lJ+Tre>2{fcNzEtz=kbXPsVbl3j}Tc z#G<&Ed@aYnD92aK_rv)>drR+Zcvzt6E|_QQ?knDtSPjA*p`IY3QTm7MJTgy@Vi0HH zApHug95;_XpM8X@h2d3V@W2Fs&8n}nRbnO|L55|wi)LHex;CsFNyTwkAdk-(NQh@O zXITb%u-4+)m5&*!BLCL|84-W@Y7?G;;xgGcqmq@|+Ci*+Ws5lyBAU|z04tbp2GV`( zAjNXxg$;3&4Az=bkV+Fof{@B0GW*0lYJ5Gbt6SzxV3Xkza|9mYy@^n%8z`vhiDQeO zdwW+v#UEt%jl&;DjZOdOy%hf?>swpS?I4}A@$Ni@dgJ?Q<9@poz)b{{`?bF}^Ujz;r2ubtfrTrn z_VYZd3a5`A0m&Q(DB*aILpy;qCtM1hw3}EJ+in0b<+}Tm`nI;IR!j}QM+6Gub9NQn zyXnr9QvA~sBn~F)M0OT!1oXm^GEvTDFY2t%g6HpVg+~P|RGD~V;_If5zi3kH zlxtbAGhc%zzK8jt#U(lREShs|FVTXDb-Lo5u@IiH=#t&dQYn~TJj*eV(VhE-8~%D2 z1_9%FU8@+D%FjLFuMy-59SK)VD27eacqFWpjmdr9qXGorx#^MOT?&r8sPrWTASJ|v zwOy(;h>zh*^Fo0w0gwI8bo>s&ixq=(H}j09oSZhn~tD5<&SpkCz_``VC@29nlgo`fu`t`^ki~iXW z{;3u4N*N1EnXQ2|V;I}WvqV=kTu+gcph0e=>CN!E*y?SmQZN2` zL0u2Pi#o4G{7)42@26k5mLXQoi-_08SlKeg{8`%SjD+#YLX_(})U*K3AnTtZGw-ov zi&={3pl;$nyF*=m{1%^Lgf*q8HC~=n##xO6xXTF=eLU+`{#L-ud{MZYsZD1I02|!D zV#(H(dB$E$Ru9CD=1cFbyt6sX97b}?T3#)8VUFM0$?C-doXxLB4LY;-KtsXZTC5SI z_;WC{*#6SRxQ3sL*mF}%IJtyJ2cJo|@W-W0NE!e*K zk038LuFe3=sQ6FG0V4}V$>o~@ZG|E^k6GXGW`J8peZ5A-Vm0~*A%O#FHi{89rGKa3 z|51p5KnsL&viV1Qh{YS{HBY*zfnY5rk$uoWTR=WX4rQ>s=OO{EawTDvr#N1Ef8Osb zr*C`s@$W zx#J5Os4k2YXi!t{sO$7tq2AB+UoHmgT!#pPzd-rh3Qgkvd|K9YBIT={YJB+NE#5XnPZZ=w){?Z$iw(wV+Q7Qm+bfsa+pSosEg*I4Z&~ zdx1##m?IG65#tf1y`4 z%+3_}Lz#4-M(uT#iJ9w-u2V#>IL7B_Be)Tn*u;5F{i3LA>faypOrnD~ta>b%%6W%9 z-;Dy^Bp`rLL$DB`=aNkUs4fj5B)#3iaW^fRecx0UKU0wkDr3>Zlzc{;hgqN=j6{JQ zUyl~h)cajMKy2I!Tas&RobF40Wgtey#sC&1fY2j#@Vk0PGdQOD zi9C`BPM=FBSNV2(>H*I#rGBtoIHwoGZW`f1dG7sk07(;Z%|Z-aLjK~ZgrK4jnTz6oxY?4zfI4P=T?;YneA zv7lu-5U}a}0@;o&iPN^uxdmyaFMY5|N*L<>{7wVAdp6K+aVzi3di(cb1JTHRi+}d} zdtkUlw98K>$H7WzzUs)1xNrq=#PT6!@g7e$%+AXATb@ zk)NsES@3U2vxd010^M*%AWPKuUcuntBtQ^lvL$0-nmpAAVsVU2#w7$8lOh6__b>6+lO|r&07o$2UKfwR%zR1m_cL}HuWuE zPQqeU!x-OSP-ua^wSIG=3?ZU|)E4J=8b%I#1OD1XXrc3c4&V*M1|n`e$d;#y!r2I= zTXRXawoR&preAFHw8LdOo}}P|hTYEv79)PIu#PPfZ?AqmV84(wa^*l+LM^tG4mrMb zhS+-Sf=3Eu zdz0OeCqnLw#1m^_@=OY{L!f&RlW83x0{fBH40h6@#-N%R>j867dc5`Ym%XIIA}xgfsHOHoy#|C2|CzOkjUAsAgEKnx z?lFk^c*S`x?<705941UqB|C*IbL=Np1T_^-_q^OP8RW{sb*6oSR?>Xm%ZbI=OWg1{v)`X$WWW*TPYuASyzPHZm%n7k33?c>N;8h5qvBP#9(W3G^9ZCciK6k zd&2Qb+r(t|R+0RJzly5}`mE_cSDSc(6{zqlFSA^_Lb%2E?smAf=Gh^r7LrSa_QF=5 zr*%oWjb*&T*q$RZ=6|!?|7g!Z%t?F+iZw}AXVXrdl-a7XPl4vWyh%}iejVI#HoJ9$ zyztc02tDVN(0`0m6&HXC_*h4%-}S@8(tV6B8J`eCR&ox1WuBNBfZsmGJ*M`v`5_)Y z*-Qx5C;+h*NSC<{r=Te=AGkH2j#q*TE9MODrOiM~Wk9^DJ?RdOjjOWDt_J*%(BWMo zEO)_>+k&nEFxHz2c=VG}?iZ#SVJLb6;K2xc_GL-xbJq(3Wjv$NjIhT{9eRs!K?svn zQ{)yC`AR#;qfr!5R%TFu!6d<+=_QEw!Bm164^knC8lJ!NRlUJB>h{guk#tp=Jy6o>&N_p$Tz_4&d= z1TbSU;wXP7Z&W6HX_DNZo2XTC1?V5pv8h!|4E)vA$xa4%o1Fl0e?~Njt1ya~#%A`R zDqlJUJ2v%3)W%?gKyA@Sis8>ZNDRXlcH%Cv>Znk4YwX1uQz(lkasIqp z)Pz?3@-OA~nNpidB&Sl5gO2-}d*Ek(PaW*FEViJF5_DKojWz1Bry7qg`%Na|>y4`~ zUh+sX$mu{_x{NhkW8NwLyuPVcP7`2=CbJ-K_J}02u>?(BL%zwg#x;mXwGdxz z{e3WakqEZF#}Q5=2v)~titTP7c9OKF^Bqv;UFpr2a+(UC&!{MkF(&s!+lZ>OoN}vZ zInZVrN4d4V#URw5d0&~+Fgl7p$CjZ(AOhZ4s6hNt&SoTL+=F3d!s(Pxn7d2$Oq5Tx zW@4bNM{fHLXUWfUarso9?Y=Mpf;;qQ@2i#B>pdH+0ZVu1LaE&xL@A2^`Z)h2iP|~*ox68CBBHy^Ol?(-X z>-sbpm&=Ny32BTW?H(%JA?}`t8S|qlTVg9dwF-FSIK2;#!3tbNai~g0dL(kkiR^U=sHhmwdO zTVDfBT60uoDh>?ZJ9+`(p-ErMo$z*iFNgZh^ReBv+ehkwT@K6c zvSlQ-dc!8vlM`lJz#DuBBv(PHd|yL(ve+F6J0Z;Vpb}0M1j+dsG`|GH9HNFkbWeha zWr?Lr@nf?eS20tpQ!sKrPLGLVcc!rluU#1-oj2LB(-ZJE_yI0WvVe5&ip%RMQG^oiVPb^FniLM)+S)I;7~K}w7AAFhiBMsq<= z>FNUZQ;W1=3&GXIziU}G%#WSx?p2HgWxtMqKNZS8 z-HMW#E75bL4|qGv13%@+kX`Fa;?m_o2-OyB&V8zi^Wt+2qDqHW*xBTYPQyTxeeWLl zJS;z^U~~bDEXIgdUf1?M*Qm?RDYsTr3A6_Kj4N3y@+*hqW62# zhA{P}boPn*rHdlv_gThqms6^ciwGtO;FKkIWJC*j!e})o;!pu*zi|#c51hFa<*N&m+*Npp;~lD{TT zm;~Zh{}B^wc68Hc@eLi_E!Nd%H-|!w&e59EKVxT;N9BMwf;EsP71d~w2ulW7WjMLO z-+#H+f$38F@X^=34pb(y_N*T^6sbns-O}b`B^teevvlgcCutA3?Z|N;!5yp0lso!4 zaZNQR-giCQl{46Nc>`W?&>rCL->4GQ<>A$kkAKF-vFpqfjHc?-8wB05o_m~MqT?tM zZ<;3P=A1Kdou-XGwpfq5>KwjnLFnkC+fD4hF85`RGK*W*0K91>0O=&%?us28ODfFU zLgaw9YnR73mHYj?J|*!zCus=1DFKdwDg<;IWO_2c1)HCqm#!h49r(EQZ&Zl_uw;UE zRZNXZ6Dv}#?|aV%oovfFfAyLs5`edu^y)6liv4YxK(za$;sN6^U5Fo$&J0M}OTXb% zqkEK%hl;qa&ge1%HI(3L=Lw`st4W=2nu-<`6TF9qIyNxRp;z^Tp1~g;LkL4GHCqan z5bRwapTJj=tec zWJ)0(Z-tXnvjA@tejwu>6&j-0HfcEHrnp~gf14Ag5KSlz zMOt~51?wP#mVI(bC9mWxjtmH-jGFnR3C0ttUrzcOJ(!ot)6hK*bg;8lyc3Kv2dc%N z!Y9`ct}6zcbJcqFhCaL;%f^cU-X^I)=G{t6!OLe`VB>{E!V)aY{qw@r_pi^33cd=U z4n)?4S&A;i)dh#%KtBcoG^sm1bo-x_oVO%Q4CsxdnXUpGBW=G1Dan|1tgFSXJS_lk zL}wt2-|{28Xe&dK4fa{`xy8rv#BP;x)4FDu)&=X#egsi(Jq&MTuho8r$&n_g(pe3C zD{C_*Nz`I_`%v7MoRwz??VzD{VW9(v)d2TW3^wo{f>k=nF|6d4Kr1#C;}2sd{^-ks zgskbwAF9qtgi#dB=r09VluL0$vCaI%{u#se+>>_-FR!awUM+*p3ywI*GcIoJIIy9= zhQBbw1Kv=3AR7T)&Li{oB?{{p@^X!5W_o+~@Qa@HubbU5YY=K0`krgam6Vbx*G`{f z8v8>~hRD@wx?Www&ApVlbgoNwYXxc0pd?N8E>E34CpB;m@V0LUWIx}rc)>|Q-NBP@ zHCXR_Qg8c>!rQyGfz_hELemNgxXupGC)}Lr)VlVmRx> z{N-l@ILN+lBW9J-2%;#?K>69#8cK7Ma3I@E*e}NzMa(O?f)>MFB>bB2> zE)&gK+yvrsP&1y<(3 zWt^iq<^)B2`!Sh!N@PUzp-MB+vAKcO{ikV5A{u@yKqEur}~)T)!3RmOUM_Sn+Pt zt4Od9LYjwv$XUCFk<`w5L`%3bV%*(WyA1G#f&;mcevtT~CpX|!{9KPEpPlCTLuBM= z7Qba-UH6Ccvt$071f%rarh(cI;A}L4GRXQXA zue1a0CKH#Q6aWUl@Tkf)9L>YaySJr$n;J9fEAu=OJ5_jSntJ7ogZXSeJ85!GuA8&L zyTtV59NtD_sF)W5%~uJv6>WdP3kQ=b?ao@*;bi!0)Qa1>1iYT*Jx;jbfOUh{FaBNH zo|>|9HFxC^E^BIvNVfx8V2(p^K0`ONk$rc|S!d;d-zzpX+RLSl>?2XmM+vg^T)Tb< zob(+0k$oHep=U0U3IGN~@u_@#;vo;VXr8s|)%WH0ejvm|R2BdToXA@Rf+Fb*kH=CG zhe$KjVio+x(`JZCh4xf1#R481lc#VM;g~!4@Nh~3_ge0&Qp7=a#_z%1isaN;fF-8 zFUuOs(}(F+W==oOXuiK^u!!*>xuhU0B&6_975)+_PJXvb#80V96@dbJh{zx52E2`y zfr3gJ-(9EktmmXMX)cb$%U)pVifw(OXLH}1L4QEX3s?~6q-mh_d6%Qa!DnY3L;5_G z`94^nitJ}#)_NTVW(mOgPjRZ1RnoT>txXcO@QPd_66F7!?{7pBn&m?`tLeQ8eV1z3 za(ZoBU%H#%`taW0U!h_?17t*UQXz?^x;I{&MEi>(+oiL3%m1b##(zt?qZJ}ITg2Gx zJcAkG2}Qm67_04OXSMbLylxQ|oQXq2yy~sUC(7m;W43I!<)(K4p^aRj^OhS=;HA*?d-vqqzGl621qN&KcMOs}M?W{;uI1oEXwVF20d!N>zLE7~F zL-dPN`3{07{i5uQc5(I4(%xatA?qcP!o;xyGP=IN8SA{H+@1|G~m-M0gL{B5} zK2o>yNHEilrBq)=^zUY0@RCX*QZ_X$4`Z~+P&PvyAs|S}c0;x18I-pW$z(^(t%7KD znjjNwi=0Ut|3DPoccEo2-K`zh#Zn)y6FdmF3dxE^@xcTO@TLb1lteZ!EUCzWW;UOT zA%B7^s@+?j{O10+&-pFVA0A;KK8oUT?2~VCAbZT+vNzVTUe|;^V|ITe${Nbd)>obS z^r(XwRi(^pw;3Nta4Shc>6Xe1k@H74NyX`U#`ozuwigZe&f? zk97{T9}V}x~{y5vW|IYol4wOx& zK>ck_{QiN+hYE>I7L{+V9!^tWPL%lb zhI&<%^SNa(Oe=w()H>oipsnz`YTR!xwQ4uO+cGs!PAB`PNwilb)nd_Xs)>yy&I6Ll zPMOGq4@fvqjyK!8^sYTeTkaERleT`^Vi%DXyZJ+OwdIXYb7Xt?k~^t>+a1o`wR4zj z#8y`LZwkO0YXK;qO?~59j8#l51hb;u@k};L>d>E6sdP*VLc`D2xAHgQGodA&ILW-m z9AgkLA{4EP^HDQu3#RAa*sL$|$imWOHpC#RNQ6ML;1NAy8U(xvo&XgDf~zO7E>u6g zRv?(>)F7N+vB{C=fV5vsRBj}83T~>jUHWePuG`ra37`J>mu~AC26AQ znNH*o{?vebhGLT6ve(+qcHWx5qx}VP?p4doz=ZkA3!+phEml(T3PhWVAS^Xm%_eOuEvRAb z3up0Sx9@Z`Y8H4WTMYHPi~XO!{k;Zd61u)MGonvfp+xW!t|wFLl@FyIn0w45gGj<3 ziKE@mOR9S>h=OD2d|WZc_e9y&tM~%FkoCp)uM?uNTK-VhsUDK!<2k~CS0^d~ggv@7 z!eNC%#117(=12jb8EZGOqJIu~YLy{5-6}j5lc=Q5aC)gi{IaQL=CAgiot;DelJevG z-ZjhnFDmLQyb|@K4E>FJA;wRDH|85q)ixd&xFXDVM;3x=ZOI^AP)CKadNOf4qzj0f424cRQb|5 z;-CHgx|*7Z0!nje-fFNAOB7w=?kP>@PYP69+8urxz)y1+>0cE{&92O4l=R_C%2Vtt zik>SmZOX%6nfV)2jYcG1*rr^@^hiL9?+=H5-@q|mmgYe&K>^;jOn~a-707*Moc!sJ zpT^z`j6z67)?zbSM4H|Rz`(Y~^ENXzUl0>-Z5g98$KUr0P*e9rg$U*G%zBk(5R48Zs;C z_w*tIBt}c~6!i{1&nD%=cMdi=P)JuOV}Dfd?n)y$J>eOLOpnsHR0e>}Ei^|g*k3>9 zUZ`rBdJrTByj3O>yypXEy+kHtTGfM-2u!#A56YMOF7RDax& zWnfsaA+-}F7q_ksNMFE2Xe+1Qy3+)J&1gTn3_7NY-0dK5LgsWYqrc4b!ExF_tRk*3 zAeQ_I;vxq<*n0E?zE3JJ*`{*e)V^GgUY~3y5U#<=vTeo3@#D6b9`dKON)f~2V@Sz> zLVxGz3;=_%w4A@xk#+Burc0ZO2ZR*NIq*ejQS*eZO?>nD9?OsR4TPUUt`yvo7noK( zBB($9scK_UPTfxxJs?P1eR540mJ~-dZEvT8~>u(b|hDX>tF9u4TKvL2m@Qc+? z3g8zAB@}c!CF-56>QE|_WO#61LPo#=-bffg9qkbB3!VE<-;5T8q>@~{CN`yW6xF-9 zVbara(aC2_XW%t=PdU{%to2VX+Of4%?$dj+2D`2PY{hDE#kFB2?pH9~YHBvM^cP=! zf%+%~0BclrR|+P#WxLW=YDe_!`F#Yt{ShXRRsPJ@e=XVuc^>!j!ACtrs~Q#-vh->X zOq1uI&3bMH`&SrFhmrKlr~)8UV32TXRMuJYK**tw&O}c*siNTAW2PP&Rpw zv^nu3{{!im=dTl0Y>;c#!y5il@L&aX)w!;F$q~c^QgN*Ltx@Yq${hz{6iHCII~mWM zQ&@fS3eTx_@4NTd05Hr@|47@;006P|dl>Q_-#4*kkkt)ziG)_Iw`5cyS;+^sYo@V3nlGD62V-Ux9(!-o&IO*wjJ0kCv+zyM|4u-R_mbmsiQA`eE( zE_e{O@Kia2zm^83=xLNoSgu8@ZDpFXk{e~wE#CUzW#8D>R`QmLG}HG3ld+_0exF{# zPhkLH_>+1$&q1{-Xgp=?Nsq@Qj0J}4XE2G5JWs`pdd*m8nGHatwHB^NX}uSa4pl1lc#{@#Jt)v8+`|jC*b4@{wGN+(I;C! zZ}5y|G8<y$*MfAG8sz|Q3j#pP-K-W<27wT7 z^b(EiX(^|N)&pz5f=ZA_gzP*s7bd)kRU-EvMC$A4jzm+U-+A4`%6=T%aYO3$GMq{v zXy3T)55*@=|8+Qjk7|V@S@NafEQ=?s5CM@OkRJ|Svuvvwo;33vsKOh4AYyfH_Vb_3 z(ymib^_ix!)iY_9RZs`gr0n!o6jateTO^HW@pa|AH2YEEckmmqz92(NJL`Pl+vB|N z9UFtC zRRXY+J~~?mBvt)(6O&^Yk3rHB2mnD}XSm^>p@KMx?p|_}dol6I*Z^Pz30w98EIF2& zgf(>E82=6^)d=3Iw5M8vxF}GvmV-Htd*^W=MWy zj|{5rfSQ5MxqWQkwiFY<+p-VP?g*F(efsl6V_mu4+i}LO_T;N|G+svr;_sGHz3DF? z*L6EjpHnB8qpsvb(u*iIAZt=elURx$gzGe^@}J3|gDUGQBb5>xnY6NYjn@;|)Y2&m zb^sXCVsCm%N}O+Pr?@zm8v)gBd<$FTjZkJZjKp^9g?^q8yMGtn{Kbx;tFSx0n7Gb4 z4vtPT$u~@Fj8`za0U>5?*7kbix75|9e>Cg15;fp$s~6}{Y|9(w12XRXp*>v_$uX4a zQ9d1YJUD5)& zpivIq!be6QsFKvgsuQK$YNkSh4)~hWd5C|8`l zK3Y5-fE3^)MK0WEs~V3WN=)&nwxjJCI{q|tD~HK`!u49gGtf>r3<}Nt^qwyv&g4ks zfiJl5tAv>=`0YBY_Kqa_LkfkM6i&aC>2}RyPj+jvjS0b&s|2ub66sbZ|JEQmhubg> z5)O82xFNq>x=QZZ%SBt!DeE6dpcNb#oTu3htQL%2V@X5>fFS@bs>4Fo0eU(J^9Xwd z-Sq)`=A)S}r z{H!TcMqJLN1>Wp@DP|Yl$r}U2e~-Xa_U6=#h>Gmd$81b96zDK`g4x~rJv~F0?vX=j zOjU2qBB2b4eu>yXO>m(AtFyT2$!^1uVV)pBG60{|{Mbs;Jg7KxnOX`J0C@ZHF3>gK z;YtnLgTaE&D=in8T+0)8_pDwG1buGu;i|t+pEZv3`AG^bSWXqyKJw8$Z$^$&OOzh2 z&VRybOX>W;d<|4ND;v_Zo9#Y`0`3)Vg#8rRp$kr5C3D?x>X4!w@EM^;=FX)jdyw3Z z3MrQu{=j^oHC7(|&i%EBYRt<0wQXTvZhoM#n{a0s1WUDLD!h^Y`#@j;00y?XORo~< z*Sa2P?aWf&Xs#?SNc?0-SQwJsO7~-wLBd-OQ_f!|4pLM<*U4w17L>K#SLqN}sa12i zh8N}|58E>?6NmUFfQtF~!$ZNlrijO4@T}o6NlZI$jRMKzN*JmOyG7u-P6s2cL; zC@Tj_Ewg!T4c`pUUXJNWtsS@3fltJVOGG}<4=f@U`7zx{pYSolmFLnzWZm7&ZfG0u zwzvoMq};q{5;><<%a9+lYAs&JCHCBVp-H5MU3OK|@zVe)<5x213Dk`n;@PMg#XD9T zXVG;Vhy|eHNiSppxOr-SyUpm!|M-b|(4c1y2i`?_wTRc;sZ8m}oC%C1F1l|f$W>gF^GEX2=)7 z4a#aokF%0PnEe8H<5B~?T^G0KhV&HB2p$)d)hajfI-;w^nIXQXAt0_48i~JuDTAAx z5GvlvWy{sO*$N)#a%wvGW@(O|@%d*Ny@}*imV9xcU4|3*`|wdFSIB*rMDfT@O3bOn zf1dKtlS+4m6lMc~uKvP}wOx1w{N^k&n=NZ^$vWbe|gyshx+8kajM%-BH3d_TyWi z=nB(#*Ybh#o(y%co@w2u^p72M4ZJ8vb61gHgRUNbuneb^UdA#XHcW%}Pkx)bjK?)S zwbF;Ax3>&lH>{pcq3Jh56KN3=Or)tE)KLSmN>;2YMn)QGPHYrSZpqdbSk>fI&I|smz~eroSff zmjqByAl7Hp&i&kYKt?dxk{kJLS6gkhyCP}R;2W{wZU;C@{%p=qsPQp$xueSWC>fB6 zF5kj0h2I_z0K*#s#dkoq(wv_U6!+7}ec=?SB}x60s7jYy{=hu#v%aM0Zrw)NjTK-Z z1@=kyBdD+?UUYDno&LD#Ei`mE%({w@Hqdn*c@2*V_*E8w_rL_UyPsI7?GY3jN$#?*&R`IL+2* z)m$vfbyKQ5mq?T#6B6qQCj^RF>SNkwmahrdYbJQrJoqZb0J=TdfHV7zn&t0f4%FS(O;E&5l zqs>jGG7KK+LNC{b?`J;03$bX-$rzbqIK*hAF1vwnrYU<2S?B%=+Gg3=;JkqBA?F)kUy-k;g`>}+<^sxlWdo0H1! z??eo2yOsS2r`W%rFCeTEPQ2Smqh_iJ6k}!2Q1>BQ>Vr3buJnB%%6V4cFVb!*8&_Bjsv4wuaB_Tk}9dohoR#wDnE=E27ortVsb7h^@4s=u>! z8vnlol17EVtO7W9T*o5fDo?liE3JN!j?s$5a#7|QhfRmL-B~jHTC35gt;n>6pwa2z zF-dX32fWGd0pt6C$z-7lhy9lyMC+2`KYYaap1HT?ENi*@R0c7v1G3A|xbShS?bC53 zyeaj-)ZhS;ngW_gx@T|S&Hu^*LJ(#g83v3bazUg1>g$_8Sm&YR+2=`R0KBbx0uydd zjjW;WqS4v8Je)-$_(v~ZrN0bufk+29JNop_BZYnzVW`MczLZK`)8@yYOCZ30$kuGJ zx;YMK8AS>51I`oaS3RBJ+&?(LTD$Zl4 z`ZP-nqvLHZ4zETj0RpteH4f=YTYfR?sMBNesyf_9ID1xfh{_v~a7qefpLW$r=yW^s zkk$dbiG=`@ZackQxI`?e;Cbd7l7Sd;iSD>*Ex|}0HilM^^B_3--vWguPmXL4$5c}B z3(C{uBhb8$Ga-s!$|b~Kdp~L;3zCspvk?D?ap@gybImzJG^Xp{c?B=Yl1GkJqixgy zT|W@F!&yZ&)+p8Dj!>VOe>Nk0BL_*KP~MSB!hSej@%lLNs3Pi1^mIUn@r*`G?1P*z zQ~mDS-ht&{T^~fflN|YATVHGr%nC{Dzt8{8V=1+nc)S)IqFsNI?#j|X${KHp>X-k1 z+J?N4d%*?SNR{f(E2ePle; zOF1^`0e~TYsj`R%L`vc8ucnpXzh}Nk(B*8qGz(pW7^|-+eL?20i@%pvi^Er}7CkSE z*$YfjE_KJ)iDv5MCKOaIHAjq(wC@pMjLqmT0buA+TCC+%Ef8>qI2xeep0zTbF-g=c z$7v=H4bq}hAFw@;qCGvygZWMezY<4$c?r7+Ag}kbd zq0)l0L%ape0lYFx-xD_gvHsT0v)+obKd5N<|ET)Ppt!cC>tPstaF^f&cXtm2cXxsW zcPF?L2rj|hH3WAjxCM8&Ai?c>-dpz;^ZlPXb84U7Evr|rzRP`Le~Ot|EtOVO`fObt z4eCbK!)t+`q5km`>GNicB@5_YQrfLw{;oLa1W`L_m3KD(Z(X#mmVVDw@FYIUs}Db9 ze8?b@pWGDRGdJR{{(Yh<7EU%0On2DGo&-9zd!dc0{X6aWIhC1*ppn;RDLY#WX=?fv zL>A)vZCKoHndRjZIi>;3+<_7{yj}7b@u6d zjw|P+3trUlyD<^fPQ!VWZ<}9$`vk0i$H~4BH+7tvQ%B~S$x6Pv(Ghc320);~xgie0 zIWc62A-FIXzbc}jK=1I>faBQdHUtIF1!SOKg#IcgtRLiP5nFwPFw1euCPp#Qv34D} z%O&oA;XTxr9&8qMl+b=7X1=7lB5Z+E;b{RtpkR5z*yT>xj+{gXo(80(l*9Oed{8Ue zpQSCAW#d3-D@H^qexDb{2zn)K?-Z9R?M=QKl9;6O&>OHGOI~@8)U-Ed+^K*0x-+eH zNu?G3Z=UndSImd9A3OTYFRoC5$fDqx$*P9r;ta12ntN7@xYBl};mN$B)(}EPx@jf_HjZ!d*0O~TQb~-Epj_YZ}G()ON z;g3$4F^N#o0OTPTGHqO}O$&R?ck6l?{_(nxWLFt%aL6&{UvjX-Apm`14Cro%CH4L^ zNVwZm~sGa!3X=RtcFi2$F0}%EvdEqOiYr2^D20 z$Siq=1gY8zw*Ol(@K4Mtj)_hk(!&TJIYVxhyhW{&JL|i~gthx%2AT_ZhvF{?8KtIu z+#yavF&q*jEA4f_ZnzEm5}Y>QU8&u`!%szaCZ_*Eg?EKke^mwh2)DHx00E(tyiek` z*Fk#SfOSb4E_er0)q)L}#&QL_)N(g-JcNa{*=Vg$lTC5|1r5bW;x(k z24~{g8q#-AL5l5L>w8+unB-K!{iz_2DMFg+Jp zz)!eP3|D+hA9Fp}aw%{Rse$zM<*Zlgg&~R2J!6wUZ9_B@F1qM~8Cg|ih(SvzEmjT% z^{hglEzsf1lyJBNg%9GbvDGd8x3YZTvsR8HfJJp@=}&yk%;h^qlJ)Fp zd9kSqTLR?kqO8c+)8dNn^hlgqG4uoI(b&9EAk)XnGp*b*FAcRWSVFhU5@c~Pw z#jz3vP)e2QD*kUfgUS2*`-$%*M*SA6byQ}2)Q-B*I3-m44Rv%mHT7$SGV`0Q53qUJ zd!cMrJ^+a3Xq8U-iS1wIP|Ej2$quimDZ;yTDb^B$f0k4dz%RG1KB7iHVy-#q}1B(r47 z-Y*mi*7E}QwN3PoP>U2B;*~W0#}>i~A3~q#w4H}fVUm3e1Q+Z1MQFzhCpMT;Sl1y- z+uBf6u6Cm!VS;BR{7x6%1=9OCE@7kp|=5v+J;R0gXDon{Dji~hX2zL&HHX(x3=>P1mra(jktD@(g_^OcPk zJ`Fd-A3;=!1#I#$Sx{YPs;B~0Pk z*8ERSQ|j_%**WF#enyNOd3G)|`pYeO(7NMx5p=gJf4g1#Pgn6)W!jlZj`qhY;lZy+ zzp$4uk6pVjyAV)UY-N99I^+2Y0Ooc~FMppSLjFZVg_7h*v3^Yc9)@?E2x)r$5w>x~ zXwGslT&MyU9|5TDipq3327OC%^MGXZf=@2m;;{JobAPh()23iHYCdXmn|IlG<>as;Ly zqyufa%WOEkd;1iaGl@Nr-A&M9^)Q>$O*VclKKyZz?G&!C)b zrBFY4wvRM8ha`={IHpy0+HeZ|`4pm{uYHEut_&MeF+bB{*$$QxSR=8K3u|V-Wi|Km zI%LqS3D-#K#Z(z~7VwMxgmtPCis6FHaGG=;gdXI~4*8Pwobp3W;vEOOI-{ggX|9iP z-Vvf8OVGuhQYyI19APA{^;TKhxcf6I13jmR{c*FDg^ z!X6d?fs!?|P(>zd&`Fb6N&G?P-`@E~3O*@s0htE(2lgoTTBYZ}2IM>Ou;UqRWNb8Q z64CY1D-osG$BWkHm2yKF>%(wN&0s?waSOUCbEizxNSikMjLd6HK6-cByb$#}P+Gy3 zw%Ma_jz`{;?hbN{B>zEqK|#}Zj9g7K*AKbIF}9mFy%|twmaEzd)+rcuJ8Q{twY+pQ6^d^{es#hy*aXLSQ%-lc0I9){gP zX~#fD#&-Sm#how01wn(P92`O{JPAgMa%N376Zi;vj z(D%W$WfMujFLcMP&I3Y^zk;M>D}ehxj*!$s>Xh9REsh0O)y=b2jpd~D*q;bl2>^)3 zeCN;h=Bn^#=U&Vfe0U}SZf9{IYS8;UP%AQ;s-U$`GP7~}3th>w^}Ony#FysH;tV8) zd&h0?6n)QW;wrT&-3^wN#e2_Rh<;!I1YFt`ED=}tC!%>Fkbqdq!6TE}DE`?pnJVlZ z417g0hbp~n`^Qvz&*IQ>#_PveyvV7UYET->u$86)#e26T~=K#d+0X1eXfE?Q_g zy-;!m00PJBj*1A4d|E4?Y5%I>J;>A|#EM3`O1~%Xva|t~R|o+y43YzHLdn&AL7JxQ zCQ5l7t_-O|)TzOQ`AaG=GCWsY{AE`dc_ybMlH@Nh0Emu%kLHh1)R4U-Rt3>jUDP@a z%t7nV35oh+)|-U$KafGgo$Dc|)eGGhC5>ci*R&g9qc(Itj!^(Foe*+ktEVYyu|#&u zTO`lU!Nt8S1ju{VJ6(jAhj}oZVi-GokM@2Uojvjf|)WCYW&!wbB7x`Ipu$aq} zFjaYqe`TrNpQ5jNDEgWbQRxPla$d*pLTHRd0dITtp!#r`iSE9}2T!*$pz(Kyi(T88 zpD0wFc(2$n6|5X&|Pt3r(@{Z|qdE}@7(Qg7E*qZ|p#TY9IIFtculI`kD z$aKuJcp%(!*5A=!Cxpu6R-lQQ0f3jkV!RJ_d;mS<=dbrDZT@lv-||(og!fPBUDbuH zlP|Dek=0;ysCxExV&BZp--S{K+%W+%xdAcO6E{Xr9oU$)~1 zb!{jR~FKm2=HqIZ}2>}osy&=pD4oupZUbon# zG#CEnO*X65xvFMaz@I^Q7t*aAz+$*xnYZC0iGuO;ikAIa&rr$U$IJI(4r+LHIcqLs zz)uu5oKzXR)(rsBllb*DuZwopVcu}jwr;GKdIOoqEJ9FNxzcHL+UFnH zFOd)a=vteOG<&xWR6kL9VE}4^avx+7PO*V<>?O$0dH#{phmfP3z8W1 z#DvzR2EBO?bu@2tYinU;2TtJkK#V!##q?!5E1?}KU%oa6(n=H+~KGCK>*cYMh_ zS{CeZm^@W3+JANCN&K{Qn+1SiJ&X;U=Is7X7{bnmX0zLws`xc@xj$Wq>rT2Q^<6T^ z4P>BfgS?eK6_ObhuBEHL0Bt5kD{F^QzVLyF$xyPenlmZK$ft*3x|1p?W1r)bBJS?M zd}|v3f{r(yoFTZuhp5&1X!es}nR%6WI4pZ&ZukHk&RcT#VvReu;YPZe;`YuBddSlV z-7`ROx}&zalkv-e17^k^VEo23lr*hbBDEGOCV!QFc<1lXZFM-*lsfh*w~jBX zmJD`3*!XtE;np9XCvs&V6U>+OgJ$V^8}G9=b9eO(^cOlfM6{v;+Puz#=vNJ}LDc&S zqbW-LlN|ebuqG70I&HAKCIEs>IW1&Q`SP5Z8^MDSHmWtYrb9^&HWa}w1F4jogPlxj z4)`IIuGs_%N0RXh=tkpAOx454vilHGA~d0w4Th6gjm*_c>>3HA`XkhB z&J$3R)3v7J%M+cOozw57pzg2xKv#~TvoB8B{eAi41Pr|PpMOvLAFG=;m8aa2?&TMr zbn8(<2yTf)i7vPMtU3(3+n&=#mF@~|_6kK#Hi8|Fwz^<(!p7G}gDs&?l8f-^D3{O4VNM~uq8y1dSU4JW_To6PQ?=(cRA#D9N$W(B#WZ(j0wvW%CjvF+5m|upL7X(bl z{eFWras>|Vv;EGRQm`38yYukE47-l)6-as!9#-xFeGeVLxLD*=wpIIL`ikCh9>b9d zfaqr~u`L%Sf03(jKei&Sbcs~*(X!pfIXm;eQhH3Q{;Hw~f6IB!jAdt?)sQjLpg|rt zUC{0SE@<`E0Ygfj39(zAYVUuMET{y2-kKvfCkp^E9=i-wkT2vB06Y1~(1?(a?!-H9 zFFw%Kjkl2!%#uoj>gd)Qd?oisu)nH$ZX|KjW`rR(MtOX>Z6pk~8QnQ;?vLXan%V7W2kOKv#(Q(!`xhV$#f#t8#A}rjB zKBi&J6UhWJv=*dZ&ZZWhe+NGxW(Td~qB6CnQ&Ygq=#-~j99o#35Pc}#-uQ9bHGpr0 zG75zwn+$@v`3&g$K;s5^N7j-(8ufKOo&naGcsSwv2`+sa3DP&D(19qt4AJ36$uCyP zrmw_3f!`BGr_#%~RX#tlncBM1_&h@1uklwisO=D|AAZH0%A_e!`Ue zM_=xAp|kxBJ_i^`eUYZaHE8P{Fi{11Ccc%ZIGwJY`Jj`aZ{y#@h-g#+ zAjX|GwP9gOT1`h~XrQmkqX(WbZW@wOjxB<>dRcRXy?#K)Sh0 zs=ax`Z=7&)T@(KV*Yjl`*_pgHX>Jz@p51b~wVVN-`Rv9x@%feqppQ{i@q zu%W?L@da~6H@{9~G?yYb>k!=Xs#~WOJ6U%m_83d_ToOW4(93DdmnAc2t1V8PbuLSUM6^?K!Y zlBbUN`EQotvvvuf3)`i_u?|) z{*_MKigEEo%(_%XoUslk9AM(WGb}~0mkfUZr51I3Exz8O9r%m)^89Q}RnzzIubX!s zY$38WOMLrz$MT2=;;$)Z6a75y3*pxZ07S26f2fy#OH3jy^l@q~vt{zQlwqi~?m{ea z+i3!;R6Y(x5k|E)mHyZYkEr)^w(mn!iaz>FhQmWpG(bvZmqyqlRT`&E#vwiGiJ#OZ zAx(}HGO8R5(g>C_$=mDU^0vy1DdXrlTTx+_=|_UXFioO2xNkS;TYR2c?9V)+@ zVf%BSq2{)eT@I7v$yRTxh`_jV=g{3>00V&N{U2WD%)6OC#6n;2o3^kG?sC>Vk@e)b zgt-Xxa)F4aKQYe@B03#-^Pz z%=Q`t!~Mrw_{Xr1{<@?(98f>QVG2DVSAWyd*9#B~YGF^OkMe@9z)z+E`Flx{q>?)w z^mLf1W^EM4-MhVybHX^4IDGwlTVz8unu5yit9ujk`Vx+`Eo;8wAi%AY)Mle#LtQ{iSkYqj!E!^DV}SHGb}0rH)QPf`#M~s7lSG zkndxq0M24;m`|*ia3^9Cu8J5R!%^g_{PP8*jh}>TPEHF?-QgQN`Z$D>Of0>OoZYWp z$F28({7vX9PIk9dbzJ#~oVREs1|JsPa$6Q>5HLYJ{dPiC`NJ9luTg_UkpViI^RlcL z2t&s|$N&)1l&6Cs=r9rvH$1<(2&qv$aoXwck1^oDfEc*RFB~Ed+r>tIP;I4 zOKZzCJ*nUIq1zrS(Q7?>UxKer6Jo;(uu}QUjp2x(VX)c(5b_~s@lQDgDU0UJQQ{^( z44Y|TJ?k(|9x(+kAmXvT^UW7WnOjcJLuFqg;lA9>c8T9-q(U%T-=v$7H_Gkg+9(a2 zs@(~9vJgH1AkD@%Ht&@a0FJ_#ry0TFV z=qWUr!N=9gqt|@?{M9rE-zN;)_s*3QR5#Tuv#2cGn}XyjM+cTgRq)JO4wfcxw)z$w zz~omZKpZR`;d7?e*qMW;m74hf@i;CREO>TeoML^|;Z)C}8Uy823bB`i50}LGHP`Cf zxKOsezLI(qAKDve$A%U`5sGYlOz|4i;G=SlHUmj8F{xM=^%3-WrpETob^ZeAa9Xg?~;ub{wkb_+#Lv9$+DK;dr}1V6^akB%^HqEKb% zRM7m%_)4Jw1V9w@f6aCF9%SV>6|uzS#@z1nCeG-x?{Czm;8A^*YY?>%bQsDbu`2HE zngAG2edlCoJQGyLbdxa(w<(6W?#&C|E0~FjXKP63{#!!-&lSApL@T(JmNG|@Y<4&d z?mSDNCWu56+y=*U9>#iF(f8f*EmV!rB9+tM%$C;7)71z@b&gPyrR(SUzX;8g50Wz zo7@l5MAr3`TZcqAYGghD&2ExCLX(1mpNgGW!P`W|XkVn&@V7A%01#F3ZA1~tAJ05{ z!|?qDP6ojP-XKZ%ZB5T!XR%NafhvmduzY=$sEIYuEO~%Y;kc@A=27EXrh?&~(YY9g zrf~B&{Z#VMCPt@g^giPh;7xrO>W&V(l+3#2p%7nODousldU4l?DY|UtN2?xPaIr zt-Fwc>t099h!YUnH?*P4{X3aDwdXcZMN-aH|*ShwEH zX?u9@qITqp1Rr`zxoxvyN0xm(t?ebSec0EZT@Gdfn5qL=Up!E;BDhC6@5)}9PB)N# zL0;;ukPMc)d;kJtyZCgcLEo?Lj3WuEWtKJ@^O+Sn1CV6DIS}DzL?}b?ILmoFZ~kO$ zp$v1ki&t7W7WFB@Wy#dI20&EIkEeJWWcV~z%>Ey2m%Om*IgP`TxY{#NkgX0TI~VGt zHWx!0>i?c=iseaIN)ASOW!TJdl&@;#M3&7@vH3GLh2pET@vo%rmw`}-{y$-#C+)F8 zYxSn&vrxqZ)vFhwzZA0zs9?md8KG_nut#T=%Dpd<>-nSbI-e+JIZN~+&gu?p(pBz?a5ep}@r7{k ztwv5`2&t)e1PF6j61Ig1YF(Ta_J3(W=Kil`bEP{hF6XHwrNMKP@r3(2i#VuVs34M` z-kT33y|hrTN`4=+%Q3!ch{;?eU41{K-Re-L9{zSMr!C`4=Ntn+aIi+rGZqJti2(r_ z#fkFuAMjnEi)=zzBq!q0du*`KE;B!m^(19RfRZU!V;#Nd&e@$)S5~TM#vh?sU8)u9J#6(oD4Bd= zp`CV-Wh8lP4s9QNGX{zPZ#`ZhaHnMME4!z=4_9-)b;BL~h(o#g!^OGv3|M1uNSHk} z_ek~I-s2(idn?b20R`tx^S>|CF607A3os5hlMLN2=I?~L)iuY0o7PARo8ka(JLG}T zy1{~!c|EkOZF8VL_Cd;7g3y|aTH-hx&v}p}CY)gT{1vkM_ccl`GwZhXkHQIxKM*F> zl2#YO z^c@RSegwR!iU48O1Z-Zcu)OWj_{4kx*rsw56Eb1;2)N0Z%dhW2U*TgkJTh-Z#P2x2 zBtv7M3dUYbfi3^6PN60A>rug&%5Z%i@so5+#cy0#Q)F?kSu1RLbM5IW5|Rp+7d!7%@s_G`t^@XRi55!_Lt0AYv&O1_c14I%Tp zR0`~{_NzzLOT3oXl)(eucy&Oy-7Db?s6UxHw_i_j+~DlKE6;#Co0iYk8^+<@ggy>89x7UN?6|ajC0#6!6Deb zhryGAmav)>;7n2|XC`FQ<(<+T$EBz{88Pkxye+2zB2bUhCbn44HYKH(uZ#J$PYsz1 z4+s-dL_03L4y56Ms>^b0Iw`+3;Td;iQaHoSG54<`3Oz(2H_ZY!*)381Uf5r42S22K z$-)4B+Npy75$*Gp^j`z{zeoc_?69_WY@(o$q{+%ST0;8Jsn}b>Y%wiQ_~!+T%cV}~ zM~=|hcu8X;Tz68yls>0YsH|(FveROor(gm6DxFEGMJU{iPE^Z;yQ8o6HizGx%)gt0Pa1K3bw+TUL%(^hl(F;@^n<#wq=A4p{2vI;;;>^`>f%YVTTjM1 zi?qj}m0c3j_tTM?hZjeNv(RuL&?C`@T?u&S(B}!QTq^lf5N$WQOgtbf=X=lULAnID zGjR_~AhE*G%ysuI;7x@Mh`hOh%Ej><=ZBx^#e?f5!%dyDHAib13F;adq*^~Bk>77l zNKx3v%g_&~?jjRQSGKf*)9THCaPdfms_!>S=}jDL^O#fRm)iXr4R|8}08xhQcQ}5L z4qJwr%s8{`^bF3g5F*ll+sroXMZL8#QU)z4$n?LUHXQbK@N80t$0Gf(r#Qo?#%V8= zN;fxe*#yL?=!kWoJG57XfPJU_xGw@i7Tb*euI{YR3=<2gRo8DD@5sGsb*uF~tLL1f%K~qTKR$ zMW<)@Ucegw07UDTFWp_@@~M5)`En!q$eCy@PQ2S25^%i$VnO)q$JR~YhvXK&BT;a) zj>mjcW177fR(wnl#SI&QCiHj|+)&1Dz8qIXk|6#aylj^Zc+*k`qJJ|$FPYHD-yI+E zm#^DqR9(kQl@rBB2623?$8CPmDa88O+Z&fm!AfE`$u*h!KX4JIRU$Ab?OAbKNzFWD=bR}-uu z5%O1ED1NSDzaix8HXJsZeyMc)nTc2>I9MyY@(<+3=$c|p+>dp>R7Q#ai7Nf$S258z z+L&+hT3EY>`Cn0_0vmqZ3czTOiNC+~i4_HfMsHN#*!Y@S3)eD*YTQ;A#Wq#M11*M` zOA2Mso~6{Qs9c)Evj^OtBU?L%J!f&Z_?VY!MhrLlvPtT8;0*tiCOuL+XXn zl5%+b!DjPN`0EUvITv2C$?;xe@xshDM9{QO*}sj~joY^->CgelXlr`SqvaJxLSH;R zQ-7suwoL)vCINxi`-U{099;>__b5t3gRY-{B7WN+nQ@aB6LY@L1982s?)H89^Yg%6 zB9J`>KAjQ33ZvWcJKYxHT zx;VUI;>+<_l~+V_j}yuBQ&Ts~bIr_<2WWbwzSb_$ICi1W?c>~DR)*Tj?N#?4s{w+m zXPotWnRO-t0&~HKR(#}kUj-)iGhtfFmQ);%7Q`k{klclUO%aVWi0S=NC6#pOo8F z-|%dnPiQ4pyh%gDOM6;WF&-fdL`%!+A>T_MRB24}od>*)X94jOUar8U>1n_DqQgVK zT;+V}O3*sj)V7EB@1ypMYq$pOs-Reiko)2d@ecfex(go4jG8*;L0I7bX1k!}IVV z)frW+zwx1323WHH1ho{FSlw0rc{QLfYxU4~f3I&5xy)|21>XMT#+l%AT%U3aX()tf zho92*;_Q5Q3OoUB3*8=LPsWW zsktESiZ9txV)nHv+XqySVZ9w7+VYSdt)SFRo1WYK2s-0(QT2w zNDwO>lo=t&Pbz1G#r|Lo;Q_qiIly;8x;828+#EE;JX?%zqY6DpDfmJF|nXvDxtYs~{Fyi<)|f7nJk_q9Mq z9^$yQYOa9t0XLtA+w_4_C)Ev-9(wfWaLF#P9QzhWpPM5bLoY2j!&F%5D`69%M-KQC zS^uUCAWbE?bksr&`@3wkhfQ}ZG=Q%$q8IQ+0tJ$^RA)@gba}Kcn7&AMR|cvh77F%y z(SdgsrISydYDYBgpVzV;P%>Ha*z%HxeoO>rc2)W2nA#@l<$R`@6y$ZC{!KUa)|TyOtq|;TDrP~(WBm*7Odb{vsA6a1 z?}JT$>dWQ4ANz86;&lGmv)m+Lm+BH>56Cf^O==Go)#62M!l z5lG>a=W>hcTJ zF0;z>%y7K%Q2;XYX2aI@t>qsRbEE4tq!kn^AL4U$eckH*2e`?oW*m zo05IRzE>JX~<&a=hM6NVNTZlx3`j<^PVV>tPJW)|9`uC|M_z7B@D_4zO5~p{CfU9 zepC?-&18VMs5#t+rzU$?<0wQ9stC9}VMd^PLFXftjjK{m9h5*%=t@+mls%LzYolC3 zf3>J4o&SJGyN10~eS05e%LAzg3Zxd=S8c%&N_&EpSBXg$sJ_XrHVp;vZ9rf5b2LJ>8db$?7*>jEA3Y3^gK@H@~CC zarMthNOAqYiI)FXSODoVxI=#V(W7Pnu&e#p%oqrNBz)RI9~>z4md*j`tqG?>v(i$A zF%%yB5#Kr%d>HT#6Z)%T+;ehe8s)6^-pJtPg@uf{X8D_-7@`B@RbkL>#=p8ZE}tTj!}b?c%!K~7JD()y41$?yI(wC> za2;SHA>R|?yvgf^CHK+z9DDa0X1OL(Dr(zr*!Y&PD;h*p)GqI0> zcU|fzSg%wmze1-lwXz4W;eO_+>e&(nG5-z07<-vHRklRAt+0p+ zX>81kdLge=dSI%OT2-XRclhEQ_fX;iL`!b-PHAMbJH|2(N

Q-G1tGF*^=6IlUKHOf;0)1_ux@4-#zd6m(vUtmDcIQEUnhg z!-{7a>CAO-MZ_GN!a@{u`OW9;Z}?eESY7@STNEc3V|MhEZTB|h*nUFy59`w&6UQyI z!F0iKc-igz29uCU6YGn3{rFUM*5U`3WlUofzoZ|_d^wR(X2e`_AcenSy(uCFSV951 z)trv!@ic_Lzh{#3N}E!kCAEh*`N{C<@9~)T39`?c@N4tMO-XxjXh9wdo0p@H5d(P~ zYVFIYusCUkHx#bnp#wHIK55^yF(o%RVw7fzdV`NTNaJ(r=}HWx>bi321jAGd?LeYw zB||mIJ3Rt2%@MHHB=(Ci-y+~meg??C6?%ahws1Zu|7p#;Ldcnls4N{fiUPqf1$2bG zK5Ij3!n;RT)Zh#GPBfsUoS z&`Uq5A-zA>fvx%lkEOU;Ah|+>#n;kLh=EL&3!4HwnLnIFszH2n>I4G<=2ZFN7C$pr zE-6f|p`%LTVX2K9u1ye;nwOrjz=&K#3#BnKshi?hd}5HP47d2%Bg&)6q{f!Z+DcQc z3T`Qg3_H9wYL*Wi1iSw-w7iLibk1@G`}gokzH!vCfq$xwo+LKRXsjs2T9a!~*gR^Wv!7?p&&7>rh7rNihC9pkGkr{PX+MSjREqlIHV@ZS?c7%QDMbL~CIHS#?`S{N>SPLQf z0b1BZ1sNV?{G%ENMiycdCBCEYA6X~z0wB;E0kEm=hI03y&F`(Q-&m7h@y!mB_PeJl z1TDJ31D{n$YSY1?1;VJ*D8tOtNPtmEg$>SZ3(J=6N|^K=`r#oIp^LTQjBU#SQN7oI ztv@$V`Otv3Niv`yIR?ACH4pnNQ>r-Ez|z=oc9^m{4OoHPV+oeN%yII*&xw-jUaO#- z)In^%;Ps(;^7UeHNr#fSO|P>*xE5=mriy+@c#syMR37lA!x8r!l1ABFC`5U}B5NTdnqb)hek6)L`q=XjS%xIJCu$>Th zDF}GuOanzwf^yu*`vZ$?WPBKv^F43}T1B_lnQ!byLC6}dd4D-}p%BSt=N!iLE=2{# z3<>TZnih94#_V)a`)BaYOVyXr6w${cK;dE=$Q^s8Mhy9Yw+=c`bf$9$MIUs-hj)A( zACI%>(D1rAsMoO)&StX)Dh|{!wuI**ZYDWD^v4p1y+}?`GYn2p0$fzO1!$)^5XcT0 z*IR!HTV`yg@tlYh$KJkD2D~W?0L5YxIK>RYNo2%P%k*JYMW-dg(>I=#2Q?Fe^9eOX-WP9;#9EjBjnXsNt5|&*$aWJga(5LKJS2F5% zhXdYb?SSGK-VuhJ;W@yrTlV%4J%%8KI9d0Gd5|w+T#y3a+Cj2wm*G+I{>*&dKKiO+ z*w0U!qYZVt0Lq&mvApxNSvwTV*%C95slP0?Pga4KtqJOn^xqB{zq->363sa6(DE z_2a<0tYfQyFrA9G6hus_>ECe)hr0z`lEcsogBAjKW2XT>6#w~s8*=dgh#E8EDzdP9 znds%gVj2CI*_HxE7pG4^&L-~Ue?s{Rs{`*FhQyj zq4I9|4)7+p1(X^h(37r+1(&~2Nvm#W#Zz*-P)2fcW;Tt0{d@Gem&1+;m}Y}o{XUr1 zOB7MXbZx_j{_LGIwcuJV0m^f+m^Kh7y<@XT$|;7thSH3WBl53I7=Z=Cj=x1KvoC5Y zYvPZ6QM%-Z!p;W*L9Sr#@#AAYPC2Q`^Qx4$Sh1ZJ7mNvai@eQjM} z7@y7|Vkfy$(RBL&c;lu4KO+C2xGEfd=4CwojUiw`UEFw>HXiH|tK0tUG3XNpdVKOP zi}AajGg1h!K6k_TW3)`bavvOwXGX4iu1_6h*=9V0vR;_dkx}yvvET5g@i^d3s~;$X z8+QXuBF?sHbrnr7vf`1vMXvP#lNhAzSvpVy+EHK@^jgvyZ}uLl?F@^zQr}?~2x!GG zVa<;AfUSdqJ&cX?PW8_@#X$?Y#`cL!m2K@^1iaC*fU@3+5k6?BwkWQInwIDN!s_-f zHOixnAT#Y44b#REi%_(=(Xa?#o7Js-@DRDsuvM zH5LELeSG)@AMj^sKN4>NA!Ud_v!A2_h-YQVNX2RMsTMva*~yCKeC%gyS5t-+PG)dL zXEdZ?C>RLopZ!I`XfEE|RwD3DGyTPGBuiuRQ)SJ;%|`mg0;}*GOwpT<-+(Z1~8}CgtCe@Q10Ql@OlArC{dc-;|B) z|IK7q15XIaZ2;crOhBbd)ydgF^WMPh)niSb&&`VNT^U*pQWGf1?Kk82qt!px+)x}{ z-IYS>O5#n+dIL^IuooKT@=anegCBWM!y>NhDrA(8C?G9ZnUW(edE4M8zdHkyK;Nr% zuhkqPUUp39z|kpvZg3lj_9GphDEbHJP1Js{s#rT^Tays9N24dL}8NyYFQhCXb6z_mL!s7-l9HU#l%Oj&kp z*Z!>ZSWf8IYpCKR)Z}r2pQ__cg}H=_cJ|_snk0OYdl41@0tZw*oM3St6_W{b=^sj; zT^PD`s5ND@{}3K-oe2`}xkpNK8#;JzktmOdPjNEHdp5H^iucXp;rXzqKe)p37nYYF!# z!qPf-2n+PU8m2b|6WJXJXXW6nxD8dwt#cm(j0r^8oNj9a~vGe2q8F^31Mk0RW$o3KK#w@tSeM5R#FJkaS ztCc#+{kHpwyEX^_fx_#~oFFZ_p7M5RMJuWdx1LHqjP&@Kf;7ygS{zr4oxLnrNIS$5 zBMn}Edy@7n)h#0RK3{;=`jWrDOcFCN%^rFDO*u#En|6cw*XI6tsGbp6!mcqe!-{Xi zZ@Xh|lq)zEz%?^{0Igpa~ z`6hKWg6L+`=_lDZ>A&9WpF%|6h|a)idZeOfh;hVJRk&v$MoD8AeBZN&7B~n3E`Ai2 z(p5QS0BxeVh121U4|b}9&Mf4mS`RH4n5xust;FAB%|2`U$el9iRT%7ohEM=+Lr6da zK~uu?GSz&~rA0}YEdYrJ1J^L@K1h4^=5)S$5=XeRYpQR8R<0d+p^yoxZ+{i8Z*zT^Z6*}-YzGr^MMVRA zsdW#Q`AFI@i%d>oc^{khux=v%FiK&7^K4cCZ?d*PVIFPU1 zg7p)~i06FrUQ0rAGZP`!^TW*+Pp>G2~SR>h(JL#*?;iFW_h~&N-_pCBvE#(57UDc zE5vwu9y@p%MdMRs^Q$_3!{I!L&hNj!s0X~QHwBtwb?Aw4kAN*5kbRWA#-8F1-pfh|(LAFw|neYn*%{PA2L;itha)PadmLlEG9KRD- zhi6_~Kz@4Y9Xi2i2l?_A@vNP4U#4xzbp<~8(Fks}`)YCd#9tQgM;?kUW*wmbUUesX zwH-my>`z4waw#CTr(ky1P^9Zg}H4KA-pffiw2lXYCbp%{7mVk`zResuVOi&4ZbcC( z==$D%TGzR+>X*b#hN`BdS||MDqh?6Yii@Uk3!8foIxCnLz1%SRL`-Cp%vpv`j$|><;S; zM{8e;JYkChe%QtOGk=4d;pjH)x;dPtOhd*)Fya?P0=lx@_e~xcQf#S>Dc%A``A2L6 z{yxZNv5WnUuwNvc57Tan8PR1b-ue`2lBw%A@4Ym?%)9wu5C?0x1R+}Qo_T@fE|-XSvF0^=6d zB91Ez^NrmY-YBeFLLAF$Ahre=BX1%}x+yTW=sZsovfqXxC~AFPV&KOnWQuFYy2#+8 z+2#S$Ocw~!ChbGzC>JgSBHpOaht+KX74n?K$=6$^~m}p;I z%$pxZ2TTKtHcfWAykeHE|K$mP@=<#hzOMOLlQlf8v{2zr-oQipRTq|3=ToFN028s? zlPW1XNp81_5N)5nT#=KzJ|9l9=~yj~{%1DsIoaQu=8PqEl6kM_n~dE0?NdPi6nLN= zWC6Ri$o)2=z+N~WM7E;iu1vk~pD|_A;3{lUZopPF}E71E7j1HcIwORU(z6mW}C|*x2L!%cR`(=*k1NG*tc9+7;Q#6@|N?X=zP`V zmXd%odY8HR0=o4_nynrikGuMSkBq+B=vg6mod>W?u#Bn1nLpTEQ=6#bE;AD8C3bsZ z7(xFeI-p%WhD_wMq;Ks)tJaq8g0q8VPi0g-1jVs(M#3rco`Ls81zJV04q|2Q8N&Zearx6u;95CIHx$g#U4Rf<&U9MS@a3#-&gNH*z0kh*#O!4YMId5)$Y; z?o=gOr{MbS)YKC#tW$AoiZB>5L2+NGtdblt4V-aIA&yIMU!`%H;bOmt2mJB0@t=xI zzAdrMg0zFt#pP>p;`Nl+ZF>aTE>fN_KtD-YZ(=U$CSw^b+Xg}fJ! zpT_A!=)HL(56g94XI=venhpWJcMpoO%Qyw7YY4#R=US+!4{t=2VW-389}$tB1J+c~ za^iH*UcrY3SQGi{6?C|~vN4toV*Z4#f7%kOa?4B3*FFG(K<_+Dujk5+3@ii$6N+0-*qlzgiR9)urjkLg`d))Y;DagOM;sSsl(<9J#lJC_A2v{VPX@ ziphyy&y|YT9nF^~Ep~s^i~jg3uQkKZS~U#ix8>sCBwZWXCjWPGE*7cCIrG}S7Rht1 zf&3&jNYs4TBR};goygFs8Gjo8bY}KXL;2nlTQNe*=%ex!a<1+Vf&h=bp>*;WFruo) zpN~ESyrZ2DRfOf96Wz?O+93iAjHv39@|U)_j&ujaxwtxwV19OtqdXO12Yi`JwsMeGmD!+alCb! zSw#>8LGM#cUvJI5=!U<maeZMQx#EpXXi$TGeGR4(qFx$Kd;_LS*{-`7RRRm-e2kjOv;@$ zNqWSp1-QbkJK&g#2%CZPrR{43OXTC5wASHG<<;@(r`$VS(o)iP3oRS5E>(oBqI4#d z4e@Pb%E0fnJZWqGZa(;PYMpSR$cd7Fiuk63p6#@*yQBL89 zGyUv`}M>gNYTVd-Cgi?Tnd zvR{M==*HuOy;0)~){B*of7pDEfA&KH4k~9}VV9POB@^AijWzq+l&p;In$(I1vTQQF zu~t_f&#)8zS&H{C#`Ks%)N*n(=%3aEv_GE-<$6n(sqvKUiq3bvM^|s=?rXCuSeKrY zNc0PURHjQC$p1QFt3|D{dQ#G)g_xV!686!W+-V@f^-OO-r10Zx^X-m(ncvF-szfLV zqD>u8IhKw&uD_Vv#WSku4X#P^lqk9j-4*v|2WG9&QX+MZZ_N{n4`l>2SBsPHbX#|2 zY~uKY%fBcL=eX!vPS9*?IW0=B#(V=oAjX03;vyp5w4wG0iZR0TYVw1~eRM>e8OjssfN1Aq6?bY*HOJTb2e`B9Kxt`$@U0Pjz z^Zz=upR>?cEN-`$lG|9^=qN=|RW+%UCcng$)bHQf10c!ei;rM*kASyNy~`y%f}PO$ zuOG&S&z)h~^arPc%yxFv=(@&yzdutVP!-HPxNTSqIAd1=B>Uo8ZAC*THQ{XFwgghOyX=u zd?r>+=-CSb{hR-R4n{UXp@p3ie_a`EPAN>duU)Y@aBXV1pHmn?a0cjB81P`SeQ2)e z}=_yl{kU!yvAmP>rH({a8*GNh=l^1oYt{tv|onbBhB zXbp)e1DTy2P#b9!eRHAR&tY(A2CCa(7&p)PM`-FpoO{|LD;_&8RL;rcziWHc^ZEb) z(DVGhSp0b)N&To@Nkg}F2n5k`3iX}C#m9RP!CJX$s-V@tLu{d7m5~D+gHB&bMh<_h z2@5>@2@2vuIL$q-N-dPr=wBDIN-oQ z{6pQ!VdAVw*bE~&?g{TFK<+2@T|kGpuZYbDmFbfj=w$YH8N$;U?q0raM@Ga6gy`1Ob9X4z~8CHxgYQ_O`0EU`n`b?!Tt;leRSX17nB~ z&EGBdC}rLWTHUxeeD3>Y^$v}G#v?4lq%Gw>AzWH7TUF2pe>VPWg%{{#<#+aE&Xf?`|7{+z*HlVlN>%}Bo&A@fxlxEDyv!oDP zu}0ojf)_XE5^=oKt!4pp`&5NE-w5J`cG}gcb9xO~|(*5r5VEa!YPnn(U&Xv{g z5fvBUQ*K$QjUzfi5EO%${kRyj6$io=UUUqP7bZ$NisGwb2}Rqth(Ld@LFIilJJ*%H zX~U$HhvsbZ0;Vl`EP__;O9^~3)z**mOjTR@mTaf`99jfo%fIa4KN(W2>brR;=4k`n zF*vQ_xf!GT(p$~7izqx5Jd$fJ)Xym7pqRC?W&#vF%3Qqm#)jF(V2;i1-MFoN{iHey z_<7Cf2kkREG3ahbCG_>=Tr{L3D@@38H|{3Ml5s6T{d}1l+(2+*GIM^NZJdH7DiJ}} zj(o3$JRHpIsCl`hbcNjOB#U0WDZzSLbCqZ3qx2myd={^-WKUKfd`)0WYEF^#xsjyz}>l4n54g0ya@ug;{9PcasEEAW(EthTG-ne@EY7Y81vb763t1vxkd-&^{~ zc4MnVgZ@cPK!15{Ocv=<23y1}2+A?4ZqQjgf2-B@8_?bTAr4&OKq0?VXJX!USKe4y z9l~aV6Ip7&e9Qi>$-tC7aQgE7Jvl2ny!aL=^B&*hqA4Z09k$_TNT;8OnmKtbZpeiW zFFTGo%KNPAiXtW;{%!}3)O!U;Exj8#$QnLN{+6RzD5Fm-S7!sFDrR@U8}n%)^}LQ6 zw~%j<35Ay%!p6rsiX0{_+FszE7>2A9lh!dS0bkD?o&B`Iq>pK1(fU^c;f82CpS?;Q z04_@j?5w+kUjL(*)+wX&B@6@qaEX4ZUwWSgiCXiQ6GI!0HWBS43DNO+*mAED%}u!6Q* zk#}H8ZD%6N9E7$h@S9*+JL7FInO9p$v(ehLuA={&(gTekyQ6f~agXS9xBP;gU#B>S z%a8`0GLa)y(jL|qO`k5EHi5kHM9^wAH*H5JUJbsqL9+r2-1|$g3xj2Z#S36yIu)^ zdYxlCAHh{EgpB~@m3vv960d%mcRKPKZ)ueNyew%dDONJbUV`@Z&^fJoh! zjM{<=&P`NQ9LeapAq}d)VW48NtlMfPE7IofR@ck_8v)@+;+Lf6(D>(mNaILvXs;`B z6TZL@S4ePg+@fbqumfvi3=n$Gt=D;QqB!o9;HPoYgZSmV4K8gNIdq$r%Xe#>l)qpgAW1*2hakQ+h|3k2?ez_DhJ;ng3vtNR!^ zqSF&<(Jj&vS&lf19+2k~xxj&g=j^#H3zaqe-*5b5Az8u(!$Sr(79QD7BI2FuEIemr zRaNo6uGY5fsK^N_xo@!;%A8M?Mi|r5#+z$pXU6C#BN{{ok+@LVJ1%qrBb9^jGKtcT zKj1c2f*?xSY;5xD*H#5)Bb_2WN0OHTx#e*OB=-adfQo(`#eA?KQ>zqb$karxi=u5- zzLtckuF95ta?e+^utIM&RiU~X4ayu}eEGyrjRQS~cV9pd_(YCKpi8mUvQ(a}z_<9d z^`s$mD>9g=2G?lNF*#bgGbgYO2cWzSpCjB!;KkqE3SvQ1Vy+caps9Wp*}?96``R_L zBG^Gt$icKnXhT^RuHOFbs=6Bpf<2jAe_B*3Rp_@Nv1}iMTCOt#ciOc(WV$o~A`Ir= zZ3iySSTCpj64>D<&~#D??v?<+$p1IR;tYNhJxf8|sgg#R8yz zI&jc=?GswTm_YuUk*bo$_Rs#YM&fintp@qQD)(!zW>Tb3%~!g^>T`5ObE%dw*{~B=lT~9KSYbaXMfF4WVK!JZM~`94p089jYq{4?)rv)in)j7%0Fn5e$+xvnH~#FXsidHPv+mHvk^52I zKCUS$lwq~YHD-0pt~9p3DnKl%Oe1g6f_=CzCgBA!g5Hvn3#+^!*f1+E3CYDkr;)AC z+%u&X_i~IlHbEh6;&FKp9`tYB1a!%Vd62zL`16XUc)^ej5O=d|OF#W30CixF7xia> z!kS@?W_p)J*ljJzFY&4cfl`*lNCh;a*HDLF69Gngs`wkm7Y+}p==g{06SZHyp$y9) zhvDX4^ap>Wp``I0PZt=TQ}0Vwtb^QTDUk z>aR54U~vF?F)|n$88a2k6=iqXHmAcRk++Tdn&&+qI|A^Q+j$iJe*I4|TT$D6E(NGT z1;DV!GV9U`jYtK^>eb}-K_ylS;08?qlZmn>QyFv%!%>3Rg=5k#@{@c|URo(}#mS0A z9u=BcmG5KKa;i%*!vj;^X#bU-|Kpn~Z}gajS;9MCkVo;pot3~m-r`vs!4L48WXCuI zd6pj%+!bsyvgt2kURdLGHKF(wbeGXkIq$3kU?$>Ng7sj@96N; z7ZCIuW@3D{1BaHla<%4AKH#TC<*`31Fu#)&apDz!-2j5%K~;0=w*a~;RZtB)cg7=6 zuO{s9u$5bIHa>ce}mK_^zk*wE*U)@1)s-S1k%;6sRpjd=5sf$)kvF z`^H7@_Ec(I>)c%}F&NYwJr~c=e-2=UllT}~Y&*0C$RnCOY!CZ1Z26l936#37I$*;P z*UDbKrh8GQI2l0}&lG!OG=$pMIH(JP;JMdzp}mTJth1&$1`j>Z)f(}w76!;MYgDR( zOG|%;2~12Fl))?#8QWCz86@0@v`-`w>Xg9zE>gXw`dqeJnoKBBdb&dC;!3{-f}lIp zALkSbvWA#Yj<8=DZSY3!O<{56zea6`CDc{w>p|`eeN}xuf$bzF2N!}<7~IgP&%Xbmj6bPb^*)VApE$XFAv2^-m9K#`2y7D|U zhd%w_K_8=T(lRIq;O=^P2%HmXAc9 z>+u-jyEw#nRYi%ry*_UQR0>5hKI1BP^s6EBN+%hcbw-^eE?*RXcXNC9KEhv zS@!#7$4!GN8)k3&&cL@q_0@LF4$Lu;s9(DaPho12g;}XiW{5kSc1TWXAp_ixPf3YF zqn^EtUKIEn@B#iW@BuucbU;Qc7Lnx;=lkC@ESFzi&_kqily=bZTJ5vvcJ-pnM)>=1 z+2;D}ETH|i%{r+HqB9r9ctQUZi=kU~$vQi}RO|(TBF51L3xJYs{{R=4C^R6J?Cu^F z3i*MLi@0GoOr_v#wIOB9MlryHuZLgV%0^vDLhtF~{b`p@ zgrZEOp`b0@JE%2iHyPhK>m78LQyDai>O?1*LWa(^8U7J&`iiz_GhgA3NT1LO6V>GX z?41$hO4z!nb(;Ri(BNPv38vkIpvp(PlHGLau8&ZQ0jc&9S`ql*q7IPRL>d*1*-$HE z?xog$-~CfZwx@i=Ldsgt#bP@ii%y*)DA=jfwA2$#0yB`7HIfVXfm7bHN*1}wj7iz$ zINu*5m&3-C$T9{+X_*YtR4y}hy&m62w#$LJOU(iO+rkFjQP3}DEiu|lY;x$FwZT+RM)Z|ad<-P`|h0U z&qkd88!zijui{Hpa6-*yX=^&49tmZ8ic<6T4UwZ_>?8xkYAXK(7M-D zM5mGZIG^{1=8ag`#}QMHIsXV|9_v?Qk?DE2qYAs+&+n)p-D{Wrp!N*K*3ZNg*hwsL z5Oxxrb{uP@U#u)J`CFD^e=~f@$IkeQM>VffJju#$glN&=H-WdTlS7gVYk(%nFnU59 zWjwl_h#@Nof-BSAQOV%@u_zOBPHaYolaaU1qadZ&;?$JwWPg$ye8>I?!2&L^lPa;_ zLkm@5%dl$nXKAeilxX9@csuU7)c6Y`?@^}ZI{0vd*Cbc#ukhv{Q|Q_C^Vg8trzB|2 z3CEb21OJ+|mb7)SWTgIzy@tFw=4yF7(SA*MNWJ&;p3dPl!O&BfU)0Q6(|IoXsh=GI zJ`4P!i&Ftk^kF@o6jH!ft6m~#OMcFz0X~?sF)A^s75SFw7;PV9chatZbHdY zV}bq`R&uJ1A$4Nu5>2oASKv4N;VLh|ncSBzdhjp~W%c{mQj+#O?MmL|ynG53a-&;w z`im1F2>xQ<4W2|QLBQZ+{d|*GlHWiUDkJqU;Qt1vvI)4Gi1NmDY%o1yw-P(YYT&2v zoR(KvNDE1-sx@2xkBz`)Lsb0o3+SCNY%yMGzrKI+b(vUxXk$saC;4i9pm*U>T zU&1QW4;jV=;%;=WlnR$Lhj2>w__O?%O7&J$h6FK39&ANJ(v{?GJkd>1@=J@q{~6>_oI;+xbD;d z9$Ll#RP$iS0->$%LN6CCRIf7PTn5USMT#kZC6^=h|g@5 z^rh0jsmb|+ATZjWL$09jm*El*QmZMo?lN<*c5=r?Cg1boI$&v6a6|w$>wFcu0g~Et z!RIx-j^62jqYuNI#ZMFC>43bdNaUdb^dJy!6ZF=xw1&l|TbdZA&svY{FFy3gUktr@ zt0&u1LcNTvMaVPmt$wx5Y7_qpKS-v)B^q$CXGb#PLt~CkmxJ+*3O+d5O@a~|e89ha zXaJ(wGa4fC5A#LhigXl80ksB&-Ft5kMDKRku(6KmCm*TaeaA<1!KJFUMa>o8`1#k2 zrW5gL^r=8Wq#*RWK95Ik{a!!AIn*EKoE4gkyO>XJO3q|N)xW4q>sLd^Usa)hAJk7yFP7lz=gsay^}MkIZLXWvQx=ofYe6-!PEo}Bk~z^;}` zt56+#s!)m66BJtFvT+Kc;|_CPOL6_<;;TL8-{j_xVUH`0 zqVzKc7q3Z{y~cDL8F+rsLvGU0u--P}gnOC*D2Y23_V<>Vr|plGT8QVv39x}IM=W&v zQy7WIH>3BCgC!I+%o$F^-3=@2J)NBb^3Nk6i2lw5sSG7Ujj)(}cgaWrtPQKN1nc+j zYryw{%wywdL(J1wN=#PIF|+0XIdkcrExLoj(s_(wG3v954Z7r>JdyWKr6k$@>2q%1 z_xwN*IL;(so1yO(M#oV4Ni=X3@$-x3Dz(gm3pDVJ`9aR6n{5F03OB9IjDh`ocXBdW z#wsH$>7vMxCjNm&m+$wQS3XIU&;2+nGNE;)Z?dygAZw*5?DQp*4E({7e8Oy8DS&O& zm%ua8qsF@RT_qm!xD)o!ChzJvLAwwQTreu39eOyX;$bk{QVKf`r6poq4D8Ek<>gUtj90Nm zZbg*_w|NkK+kNmI#$!pmgZ;v>GLtkU?{mDp{gSl8N{CrN-5NqV&j*2O9}on9I8(Hx zwTCfb!!|a&Pk0g|Dg9+(P~4&pN7Ad60N4!DIzKqM_e?Z~l9d0=IX@>ytz;71$W3z| zgX8?pI6fhqHo(5Mpp@fM(#-|(EzVYLtKU? z1c!|^4uW8O&Y}0wP8o$(%TOvW*For~04M)9-V7UhKE4eXo73bz$#sV#2Dz zBC;|x00030pT+pk0{xX3n-{bo4Hw;VI7u8^K%IKeOuERklmz5Rh=S)Y@PZ=vY!{Ke z*V(zTu1Hx%FPxOv)-zMhIMLd?w2QEfJYS)Uf)oa|Z1WEm!9R=jU(6*_8aIx=qmXB- z)NVJ?IYe}+aA5c{6eRJ$3(pXc{5i8FNlFo(lQLbf=VM$r#%4B`!h(gXX}sUE{8xSZ z1@P+IEe>$9eER|5cb&x_ftTT1*o6m;N(y^KY9$Lk5s-@=S&ytSaNq;Gc*i=P@s7pQ z*gY;4I`fugGcZs0B|$VZ1AfBilFRETKqlP~wLU}9|7B{Gr#U?bk0bsE@ZZJ!FYt=L zh|X)Fus4d{1&9U3+34`k-MR_DT_6ExTBCQ;35CsFy<54f%QJNQvQlPcLNSgZ$WG1I}2E$DEAslSauJ?JJz+LU7f z-^iq2L2GvTv_5B0)p+O7ru}(8&o}5z1)jR*Y}C)1ta|KsX@juE>%=T1EHmD1eK<}) z>pt={2w(k1s-r0T+aQQ_qxkQKg_on8VTs<&yw^Q&kZq3@SCP9A-Z2WY2N zIGmDW&eE;Q77RC;Uz45Dn$OxZYA{c7-h+hmU&QSN&pUNvi=L_8xSmHFHX65D-^Dp= z7#22T?qKR?pvJB+cik&Fm0Vk`fs$D0!^i{XhDl@^-mj#YD)xPrV)c{2Uqc1DpX-Vv z`a~1y^>}EKQh&Oi{)0^PzrV24Z8(#Om7}kWwEWDZiNH=&W>R6`w)159O)|lV z`ifRTV)4j~yxu}Edu=xA1l-x8DoLQ$^K|M*(;tm6oN4(K1ru?jB-Ro8byK=$Q_@>)ttZv|-|fEb`z^}%Ze#o8eo!_VTNM<86I(>d2|%O8 z3=I98rjSM0+ukf}dfZ$S$VmQdwpM$wT}_ovxG}|-+1Mk!8`bm)`cw-iF^+~E4wt{( zY#Lpf*^-=cF!B8)-(AB)de9kDV!(j zRxBFPSIFlV6eIp!JL0?Z&c>TrABJFMgq_lk$qfelb`0rp@Eus#(TUTYgK#VZ4qwr& zi%@M|Bj;PL)O4lAOo&a~8%ZngXl!nYh+r9tH=aDtIug-B_Lphj@Psyhw34HG^Ik+q zx#5A{o@wK9C#OFY+2l+L(vk2X0wVI0;Bc#HxlKrw1s%W$77;_5JQN&mwd45FNy4muZDeBz`FDmN}7{R^uwbilS+$ZHHy;eM+b5+~8= zxkeEi`9HgR13VLj=?uH#fVqX?X$tI21W<>h#Cw4WeB1emkBj-kLv@A@P0SXB16W?je zTjwPHQLenZBL>Mi+>o#Om;-(T)%}(Jm`XCpFU3{Rn>Eml_Gb_5Xe|P`8`VV8)R};o z8+50ib24>LmoV^j;-X+3JlBvwK$(Vig{}z5UIye=M5R%f!D0R5?;)OkTh8UCvs8RE zjtk9@7kxQ0M=30%mW9gHkiyvT8r@98TmB`}51$s6KT z-M+hIYv1P)oMvPHWpXknjIWGi)bb#mz8D#W6w^Q>b}vUbZt8=&@u*$-p0zTcrh&x|qG zG~BzkOE5y%S9cxFB@H=!BV)VU<@5AVCF^h;IQAPBX0itnI;tM z;;K(>S_c04>c1s^!59u_VkthpX?|?Ilx{K|l!s{ zkGuq@M0U2=pJg@hNiJpyQwwyfJ-CncJf@KSle>sS^2?(#t9VdTI; zPOS3V7UU!25A|PTi<$)WP9W#ml3fR;93-j)hyQl7P(ly->x>g?!G%Dy5BSzbEb*97 zA_q~_UBWB2^fowpAtRY$E4)%r==fJ%*>ymEBgvr}uy^^GTpy{RkgZ;Q#v=eXdc+Rp zOu_cxkrAVgmwvc%K$KG=n`u$^XCcg{V%VrO-t2BMeiz+xH!y+SoMyefJUSxVUFCvc zoSASg=-Eo#{O{Sfz7&-2e)op{B;-U%pP4o0$`X*G*&=MckVkaBM{KCinhzA6^xhpP zYYaP_jm)VZ4oGHSSd~=C8x#PUr7cp*{RwWW@bZA{hDo{%V$CpAylME*RV}H~_0IMi zDSx%Wb;n+ObiA~w>Kyu(e$B=5&MknDwo4 z1WF*;L@yiup(z9dgsKs!!+iR_=|pLbPcpYxez@JCSIc>WfHsVTkJu;M3;!F3PjGu? zpWDtN)(4-`;w$JMOz`DLit4OFNh#9c4yi0IRDY?PclrHW7J-3GFYwzxZhK&;n`o2_ z+`^6&RXRz@xu4)Dj21$J@V5gMn&0Qhth#>~dXa(p+ff^su^H$jyx>WZG$g zS^?$P#Gxi#U|PFkQ9dg}q2@+kLylUEhIoQ^;7QS|D*cq-LTonmf;6K#l9W+N{=V;j zYeX=xc#@ z99odi1ZeblXzcW8(c}qLh&-l1Q`r|tIa}AMcpP#;qzCb8#X}a6d+x~(?Ui@kR?Yk# zY%Ko*f=g(fpZz*~bs~{BvB(tlxp^@ffi|^`qS%fd2Vzg>UBVtmEoHNlc9f}wC7dh$ zplphAD%qJvlHxa|b(Q$j`4m*@mNqm`6?4|S#;`QPJ9Y9NZ*Sm`e`_uZXf(lTI8}N7++ikVKnayY0>iI?T3(RPV*TWb z{Edy5&HmnsD7hk*v0h+XGh!V?c{TVdGE%5<%S5Rlc6__?VCVWgDKJ^@n|>o0YO#g( z+ZlOxKRS|-+wddaRPlnBa&AU$^bW$B7%+AkNko2moBof?4Z9_XaDUA)nrm#1HXE%j z$i0Y6VQfR=aQQdEk^Tuwy6n#PO5iOdpeYmvW@<5-4&h%C&aSJ|)l|8SzG06rfin!s zEs$#>RL%v83 z_Yaqb>w(A~XkNgP>9UQ&QKc{aX(I;b>y@EI=TSc?GN}@K2miNT27?tB2je8GE5^a~ zBFM9>&pWC#)p^9|N6)`J)UR32CxSx6mPwuhb+0UC)@Nv3#}W2g7ru@eP$U*7QZu!9 z{TleY*ATZot~K8utI1pMR4q7+m`RRH%+c3bV_E1)FD20$(bEy`J7G z(0+TwoC!^pqi6+CrQPo#UXOapz3d0y&$?{p>bxOkHTx5ZX53cUrYHcU7ZT>qg&*<{ zInlei-z*8ffvBgtf~RN}{$!d1stFWoI_yVtGzP{UJflj`V7_}XN8VG;~SSVu*KrLm3R?~i4|OOGE* zZtUD)+z-#3GFz#z!aUC(R9qgS-{z%n&VTs6^EcrjPcfKYvjQ!Cp*MsO>$5xV9_Bqe za0%KE4qtDHSGQ_whk4r0JTNrqB34BRNe${x?7|5{3%7SVMho`htFmE7 zdG(%Q-_8)FFmgcM1=y+VR0&?}9ARFm&X-+*=lR3WxxH!e zk6p#D7`jKC*%OTAB(!}j(Q;#PF)eLB-uL$5ZJ(%BB4`1^(a%U(OXH=VZD1KX%wrNf znhm~ECvxHV02_lHM%ACZFM6SV6?(`eLODIh4_ilSt^?4qb=abI84loJ1Qk%U2iqHA zY+*li4`67QDPO8{WCUB{;}y_&*HA}X==R|UC!w?~8%~gJ$M9%u8LD5X0z=#CvgUez z=EQkuLQSMEexI2wZEWGXg z_rIA1Mnzu`!*Y5PXq83PuuWDI&ioypO!So9`W{+&dnFL;d@^iuGvEwhHHiWlgxV?TG#NnsZs=Tt#`(Y4+rph%4a%PVY-LACYR^HY>_p&@7FdDx>^wD|1 zlMS`CEnEeeMM!HMFjh@0+tDvP(ZK0 zV+MK3eSy>>Ag5kkG0$s=zJv`jB;ybpzMvb5Yt$3DPRVFSAo{#q-qm9GGRnN!jH%{gd#;f#^lanXIG`;@ZXv%Iw3(BplOGsFV=M{d5n3P0jq}YTkY+pW2~1Dp#?|j}>&yo+#Mt z${rDWi@w~HaSTC-g<;e=y5hN73vTe1_a_x{(_$7H#;7h3vR}$|0sb2Tf)NsZW`RF+ zY5Ps4q}YG@JBkyYQAmA`6gXj)qS2?9AWe1G?h%*aIO~AJ&gg?-KHWMRwY`9(jOBhf zAsx8o4bCqu8t}4TSpxhRDnw{T#+fFT!Vi4qe0aR%`ZIj+@KsutyFd{n$apBTO|?7L z0Q($e>5J;*=N|$WkyznD$WBd?(*sZASZo?RDE&^cXH9d7;;=(}fkj*&O_bO{A}CUE zZ>53AM_bWJ?kR~rw<`f&nl)><*6plsoA$R?F1oA-_UWQkW$T|%o{9d;2G2w z#yDBM#=@_C$m44C&h6XTk(^OG*XBqMfG#x?;PTX$ELJtYM!<|=)dI%|bL79|CUp)~ zAz%hHVyOjujp8#rTq2@&hd7qqT@q+#Uk+^?mEc!v)y^sYLxiLms5tx{6>s$@DH(Ta zR7qoJr!8X;8r7whg5eqONe2agkwMjpQ?fTg8c6QcS$bM8CG=*Dp&?!$;^Q#WFO$A5DZ9#zlX4+t6;N0rsi4s~u#LXBHw8X%8bXK~X zLH;DEgsy)$=~UMxs1Ck`*Z8zGb1n{z%mn^b_0N7{D;T*z4Yy$pL*2s!Tf6eA|A4n@ zLX5Gl&mMoI=i^6jCBH(^=)qArbO6Ldyei|o{$p1vHrtgVdo+=XD7t-p==2Gnf^iL**m;q+j>#!acz;!Iu9H&1rD!NdnB&^ORN|sYh;hOpYy^Hc; z`jbms8`0h^%?ogDxm`Lqh?@^xAsD3?U#XWjRQBw|Y`jWqE^p8QdmdNJ7M~dqcI`0| zgY;8cp_*q5i+hu1^WHE+_8hWYLiU`IC1?G3GPU-Idf5xy-k?BTP9dMY;P+j0f9H=v zp}Ma|^_rrVT>Ad|dGxGjU$^)bzU0 z(q*~FhM=R4Ih{X7dm(5+4@;$l`(F?A#xiw}J~S${WGWiDTUVDv_9>Tt&u>dRI3VvV z{nV9Zel?%9?Kj!OMrv~j16t#K4W_GHLmm~*IgsNuSe$5&60h5iMcfX>g6NM$C`~I( z!$vsyB6rO0(CH}q02~>|k(P`nkY&>=M27>sX{?xyT9@ucjXya1mr_GPfNXKb^hw@M zGZ#+ZAlRsv>>*}ooWdi!GtVjJ{=<7e@m3blnsYHX?KF)~CTO~KETqqAB4^C`5a4ay zX#U0=2FeVKke-L@e?`v)h{1Oe8qN@rv_TrJpQC#Nx{qqQQMj|&377(OBEsS2`$+u( z|BbQ1=;GvI1A1+?YDA0%ub8hf*q*bB0*C*=vMR$Ftm@9yxep8bZN;LtYcX_*EG$g< zLFSB0e(SqtUuh25irB+JJ(m$N_8Z2IM68x|8(ArPkK(sOqkps`*{Jp5@j7e`aL_&C z(dwWqjlFGOKa|H#hgpCFj~00iVkQN(+qbPK9;#h z$@4lIP;O=qI6q*Ad>7!`*)fQvtw|)Rr|nkrp>2y6+F~^{CRPCdKC5Z8!U-gapxXuu z^q*}g4yx!lvg;$}D(0CJ#}f)`Ea=os$jKK@g`_-{<%ZT(alk1Fc7K!yBi`L(t8k}4 z1ef-iqR`AA?X3HqPRd72H|v`D6Q^_8N}MwYn?_={#9AbcO~lNpNeH2523 z{-z=F8fl9DgcX@$Wf+-ddkB=G!Mxuw|2-#$4{X?=$d+7y^CHl?drMLkWCTWeL`;ij z=Kd4mh{K$TuP*)G9|IUJ{)RZfhxq_Y&9iR$?;&8oPNex?y|S&+eL1`R+gqUPPnv#0 zHG0;qq}(W{FMVzo-zBu8Lc(xgkdD4$)ecg@d8-STbvyY&b$M~^0X{VGtQ!jo>*1vh zhu?%i7HA6v9)>m`MRlqH7*Txg3kX*3HKiD|>F4=K*L0UUUZ{NZ5)J|}S&wmMa~x!g z#!sL6OM3jUakhPDs?y_XzuMj6jF$x+kLcYQ=+B<^&1du@06=$I@taDOc3_0z|LofDfkfq^r z=pa_7Y>Wgk3+PQFs3bS7X)pMFw@lWbT=nc}tT*1F-{Oob!|kcmT;H%#P0v6+ACIU} zt-+OBNC9rzc?b9i33YrtgmWB2H<5wYmT#_Fra^!4iezOY$pc3Zs?>fhy8(e!h`V3@6xzaaKZ*Fr-v`A6iYyol z{g~!lOrwZK>F06lq@SFKNqnI%63qo=A_{Z&kj>N zRIfJwTTn5^YF!Duwt97ah(qmy^kNwc=i^ME${Z4DZ|m&*43 z1>Z&XFtrG;Ez1G;?^XhgF9zpc`2O3>y_5hKqGL9!9>a44`R6NKb6Z2VPW&c(;P0fo z<{avN=cM2l!nnfE77?d&3QP6fUvz>V_8EbhMfDfHE z|9Ske+n&vE2U$mO=fjG)j-LC*(!q<}tqaj90wl(av+l7X;nC%zFD-+t`I2k8mJB_T?ti>l1dgG(zCsA$-Y)Kh`eoIuR%-RTFg@Wpx} zM$7@C=vmjNDTyiAUzW!2ogLtm$I61%u^i>2{dnK=TRA6`*Pbf)fb=AViO+SV9O2Y{ z{QU+rlq6c39e8Z{0_TIwU*TGoZ^(XZtho{jCh zYwbWIi3BE$=OnJ{zq|Asy0C(UehWnDSuEHRgiwjoZ^4`|G*FFk=bk_bDR*C@7v{yS zrKJL(lb625cu65@%~kPf)LcmES)PHp7lC{SA|RWTn6n36@%U=okV%^)lOFJ4Tp&^r z02-XJczmS^IXw>#j~|V>Ay)c%8@Pf%fM`HA2eh+;pnS|Vz-b7W+U`&O%WW(S`jFPn zCb`X+1uuYRH(w$@GRmYfoUUv|?IkcBgkTXm$!7%rF%<&5ClR+1%=#?KeWiKfQ#DSSEkFLgbOh z{K7CyVpRL8hwaAk08B%IV6<8-{#1VO=_TJ)MDp*os`X51z3cCX{XTCz=4uzsgVN=% zh^{U}#41sT?^}LT$|rtM5wE8dC`+p%jw~HQ^{@64@djL^d*kvlM6ZV!7a${5mJG+!2B=wcAb9oV42AHy zlljjA>X+tQ@v(MtOy-4!_AxShaH-}r;|@!b1bpipXv;_rf9>P<-M*QTl(s5`(=u~O z(dvWy^O^KCvNxrN?Da7a3N570C8E^vX(ZGz@+rt=q~!~WJ^sqE?gKxVc9@&zFH`7~ zJ_5=%-Lww9dNWbOU9*yRRlkuS%{nXp3vVpv0)iuUfp6Su^!+ zac@PJ3{_zI&>w}T6x<)^Ius_l8H-3W7k8KVkdspGKw*?xjcnD@LH6c4*WXLuleW|i z<>*G6V61N_LHdr^F*Cn72r$-Os^1wzY?rXQyl0d*Tv^L~d&mlFwr^qg*W*sUrUn&T zrghEui-r5F$e0J!z;6LN$dT+IG$tK?6RCc1X|IRs#y}6u(PzOj)q-HZS)R(@ za7XW4J8s2+d=#{tBfru2%iczSHFrqEk&iVBQA!sI@jNi)Zih(m=x=BDoVQYWq0k7O zo>y4IbxCmTojA0 z2OV{~+Y%Un63LB>5R))>9E+f>DfWm7f4q53;{$z#<-lUP$Kx-)uVOorp=+f^1}Xhs z=*+TK_`PL*r#H<9?x~5b$Gcya^Y{?$dIBH6ch`0Z5D`UMKwV^B zOX{h@xn4b1xU9=-W-Ot3#^7K5@W@$$ooWj){~Y=M-Eb&UPCt4>_#hFG?qA=7%{|YU zDMDc0)!x>Ij{nr;7lrDse!V5;G_lH`twF=iD#t>)b!Y(D-xq9cEY3r(uiFdw{ne$g zdRufQ8BLX~UlU&WW5KMX?aQ$3%Z+sWI=BBbZW_P3JB);nvQmW@E@iI4Hi%0)l{xD| zKp5MJ%q~aV&9PMneYN^R^yuoC+*UdCl-bD#C1SxE81RiXOu{-j?mdv-8b9U1>TIIh z967Ee(Q$UonLnMm6ECgbw*#7hAabHGKp7;&pVZJXG^PvCd~$G*lag)&3AE_NV_pSL zXT1r5{*khKoV|fy{38^j_6kp6akr4S5B zRxwej`O$_rOIpl;{~j%Zsb6>;f}q{nepeK8jTAteiIk@Y_@9wj&5oQMJ81zY+-!rD z8_O~xt-o)8HVm-NXr6n2%8Tlu?AOa~mcCEDwdxX-FF#xq$eB_WZ%uLdieSJaK06Qi zFwD?^{HiP2qq>mfhy%37;Sc4k_tQL;uvq&7S4VbD&Qc>$qV}}D3#mJL)QES(b%?nF zRjwiLa6W#`AYg~YjZt7vo)E$)$K=Ab?9}ecAu8~(ThSyk1=;O5nJE}rY!&vy8#`09 z`1R$uQP!8u1`Dk=99Eib+V}TSiXf5`X$Aq;;K3 z;ikg?`0phhn3kk+I)QQQufV-wc;05Wv+^Kf3_g%QON4eSim}{Zpro8$Ru3Kllz4|r zH}cCBDdJN)X?-kzi44Kf`#3{`C+L3ch%$c*>OQ}0rc4?*nE)*yBn;Mi-}jp`wS|2- zG8eZy+|#G{jp1r{~Zih+56o{-U!k8d91w#&N<0FQ%+;_vrTXV*~dblxYK00 zq8SM;e5b_@SIdz_CbCR&8TZzV6^SwmgJ;T_O-%*rv1g{_w&VrZAth@&EG$WpKLhY# zw4qBwjlg>C?e{Vs_2{}-X)`F8ieGb0IyUJ8+~hBQ7DK>@G)rdZlDn}{mQz^dGOIF} zYca2h^+4l*JzXz3{WN_Q&Am3PMVGnxi3on&<$cF*0jOscioa{6ScHSOxQkJi(;VPi zh}(v{LnqhCF9=l65=BrL`I=ozb}><5DC{%8iyxNjkxv;x8P{+3r(^bxto1daAxdXj zw}~fNvnmi2)eQd;{n%!>yrf}T<8S7^>Cc^ePyHvHW zB6s=`Sml!He&6}Dpd1G}$G%u?qUrpKj0bR^5X8Xzx_=nYWh{IV;*j0i4aLY2!{L@y ztrDLjK{l4zD(-*CJ{)=P-rjJHu>tnnj1E2tB~)xn4bAp}Pay`NVXKUo65^zWv8-`G zY1HY#4iB&BCLJX>f15ty1zeV}#w5+ES$Y7{S6?-9Nu29p5Ju}&389<(fOtHGeB3!v zy7#~E79=J=5<|@NQaKw%^iOGB6AGLV9j76_L3`Xr+5qj*7q5Wffr@b?Dsz>CE~of% z65x?KPtsW`j2i09)f;yFW{(|wS}U^CYo>PhC&}t$^B)yy+OihP&EDX3LSMUvi>@7> zRI23)*+)1JE(&)E9^sF5z;rnCC)@g<)l8U(fnKrtgfKt1?z=$%*aUD zOu(do1EBPi#Ad8p6iCcNWVg_)a_vlC+_lLE$icNFtpza+7Fx2>3yzfV?Bw=DJ1fN8 z!B2j~#ba!9by1wA4wQ3=ehIZ|}!W^d-&X z5N?m3du-2FI|)*n?T18ftrXO+&(mAS3K(vxJTVGCHjdm!Yn+3%0A1(LL4hC`7XUD zfVYsgI_PpKzDFoVoL(#BTLLqfnTu}X;)dC|hf;>(AQkNP3uvO5dVu;wX-%yrxwli+ zOg{zs!%PnuuJ6Dlf(7qoB#|;&X2?F=5BM+`vnHzAgP5)#X1^#9`Q02N z<>hF_gPeRKWcv$vbdxvf=W(Vuz#Pg4>D~`h;!rZ9AHO`(k)EMsNtmXxt-TNbWe`^D zR3an_{Bnha7TW^w;c1eMSsu{2*sLV-SPPAs9EQgqxefWBrW(l+61Na)1l~5#z^9I6 zYLlXicB|>S7(gN8yU0BoC-(`GxFtomka;CUCxtbpX?|C5t{M*N7X#qK-Jq@30(!Aa z$l?6=jg3%SD#IJdQT-sTJtkDRFFhLVq86`6oji>#%lpS@Y6Ko&{<4b@3alqd0)I>( zPeMYA<__scpLK5sx$yIz4Yd(aZr}TWw-&>0dNZW>(lSK@c;}hZl-9^EnJ_K+9S2Ux z=PdAN!sN5M}wez56pBrii%LuJBLY%7pbO<=af4aU!hrDTao58 zR4Yk5oRmAQhJH_nZ}5%_!b6*wA}_^oLgQy^O-_hCdwjh`Aj42#3T@aez~Z81Z13L> zP{p7IO3oLWXu3F^Xx1dkf8~aAw&xv`xM?K2)x`;Nm1%=vku8%387>zAALboSot|>m zv1FSbm26<@(|6*j>xqXy`2*nW@d!i~ zD<7@7=c`6wNhD8u?!Kiuv!W@hv2W#n-%(Y2*Wz1!?cAm!h9N_hdeAlfql5>nBO7J8 zdC8j44ftaOz4ul7CL5tR4z$9%4%Z%A48|^Dn{nyNo@A-bhC&xE6i-K}wekEhI*|V_ zo+9AGG|VLz@lRHLGi8Bex`zSkpXKK7&!t1{y5Gkp5z?xjfEJmNKkYmd6Z8o163Vy| zfBclq#K{kU;5gDJC9f3oI`mkcqN-D_UGp*{llfkGY03`x@Pxs2&$Punz2tT5)eXty z2k}y|n}bAHwLJCiwLWm=sQpsT{xs}Ye7+_co4U*}J`E}McL_h4c-cTOxE>M(w4LG0 z-&Loig#L$D&sQuA$`r@zR;bPksf5n^1stN0LR>C&;D|6fiB0?EFx#t? zZYNHs#}1n@?`A-88Fc*x-;%oRE8UTZiG`^A1gdYDBDq*10pP#St-##fv{XmU_gUEU z399HmamgZ#`ZqI%es!3Gaiv)htI& zl|_f`=d9UiirwG+q!b{vM|i`QP{($-DrCmyKD$55 z^VkXn_%KEAOdI*;gE5-FWMLqgILkX9f;bive!#|0&0zb8)2V_`JBM$!?)jLyXI|Xy zh}Wby)0t0zDA(F~v(%!t%I*=gKO{AXVmt5MYUCKYNX`rRa2eo*6J{G=?F9?86L%lN zS(i%mXh7Cb=gysII0wSVt}i9RCP2wkWAaZMsn~yd7GAqLuy!8rUZqz*nKrCK(cS1r zlj*c2HZ(eh){J?8`}Q%nc&l4v?C@An-AiRZ?a*6n-0qUDFAwhU0h8EUxShSqXZjW% zB0a2k!uFyW7kg(S_%olPo2U@)x^z0DMVuwZ?-A@RLO`HhcXHpkcym%8N58cgucZNb zC3<1r^^sk(+^@BI167k}vdKMa%DxE3y6&OnzCz0uJX*IZmPWpYa#emO{KA?6V5xuR z$0k*`$%s-?Pg*5Y)Y({IER<$KSHWUkF&{gx75A4E>hF z+<)(Jl0-4c69G zM-*eI($TVw82(@^xTR zws)qnxo*IpwlRfYOXq;`T0|b0CB~rN92{&k`q}T&$y6yZ%@@2l=ziMZG;====@(SR zF~Ku`vg*mcNa|7Z1AKUa86?zn83$UEx_#FxFs+5nsTDsB;6ZUXN9YH#Hb_!;SfE>Dk$oG|q= zqHNO|`BOti6wxiFH0`1lV+1<|FFqCw@Zmje@TJwPR}i`RFezJD!7CT~w?@!UzkkJ5 z^gp2KYvk>OAtYQP7;ExgkM6J^sYA^BrH~f|nIhH&r@E5w0b0di1Tw>%cLX@aN_;WQ|Kv{htQdfiH!i%3o_D`RbO$^9I1T~TLP~p67f}*aVIkR6s z6y#jt7r1_DZikX6FhtUM1iAq}G@xNaGz@Wxq)+YnwmxwK2`s3TkEmEyluLrfOhOn4 zVEg>$eS|_F3mP3zCZN#~m*a3<p)ZZ#hMrtw2*7^>9MXq@eJh1)I17%>gT29yjec{khxtHQO@ON+_a(JAT6+cGD z{)xvfe6B!CU28&-`a@x{oXF=w5v3~?&F(W@5qCgW8z)>2A>Cg9`}Sk2@CYF2Kte5NpPr zi0?-1fgn2A_X9=i>aaL?H)EvKITF~gVRQ%nV-YliVp$29*T3`vJUt{o$}uuhIE_f< z>kE<4Ohhl5A`j@Mdx2WOK5%yHXKADl6d^dI8tDQRJWL#7orxIxAL}e}4_DX#Eg)cq{TS>n5|Nh?>UTz}4?+VzTtCGrrBV4{W_noe z*;d%QpiGE!lbQEC6HH=?n<(67pgp&W-UzO%<+u@rL+1 zBztFp5I!6S7svJWU5IamQ#^jx35Mj4A!F7@i#4p0I*y?FfVp^Kv@UsT< zN(dxiL(CJ5r7m3R-@eC79kD`n1!(*xdHCkw*F&oxw#v{Wgw?eTv@OMw zg&*=PNM|1Ukb&liE-$UZ!+6iAP9k6FKUN=!sGF$6S*7@Zy2-;PpBp_OQdBqIiGm4~ zxp45FUYMtq(f0`SxS$f!d3Yz?Ce-=t+RiIf>XDtnCA5CEDPr%lWSWU3!rs2bM!^3Y zOuX4g62})lE~1DfeKvz4l2Zi!a68m5UmSt01Tt7?{8X_$57iOf8drL)Pq+ z-eMVnc^Yzymx?9nty-QN(nVZzqN8b96C5VsLzkSoV^YYc;;Y!Fi80oq`g%fli#*{g z$-9;REF%j1#Bi$C*Y-4UkL23&$Vzry{`(@&JL^k9(JwPy6wDhgj;=(m0`HQ@Ly0n) z90E}b_DH~o`AHg$-(T9}XO$bY&bBP01fyV1go}#W`X2%EJzj;1O>h~*vSi65e>IQo*QnR^Z0cgx{1m>NH_y0U{?MJdFycy^4+i-n|h6I z!s3KL{?6#FYwmqyFPD{mnfTxLFmIC1KvImE-W@g2uvY@q7|FFfJZyN-^T&?4S+(A( z8nZXU(Lva(ij?=TpUmqL2V&z_#e%PrX~}~#4!YG2Q?zM^(t=aT9Px_GVySiJ8}C@D zk>aU{YkB|g|BAfPo|!k+67~|aUsBl<`@mrI5(qtnf+G*1szRVZbfoed8bOPJj;&xU!a3(Xn3om&bI+ zkaVaCOG$ij1L?JOfGcc7m>0e!UE0N3Y)i_2L?2^YurlD`qzOzY=R6t4t?+jr?>-l? zK8d2`96x_lbhd*k=T}crDZL!_%HI+`n`G8ZKAxm5KvE%7GcCoFXU6~(9e0%N1$JGa zmnCxLFA<^@0aB6xb_c{kL;A&A)Ji?`sMNt0=(F*)N2ia0m!mEwOTM>hnQ8xd1l)-i zTep$7Voinqs$-GS5J*!e!q7s`^F2mM&{|-R^QRXW#F_t*b|bu(V8CW+{x_rgj*>eb zoA}oovK;@yL%)@Dz=sz!xscZFLZM+I%Clr#o%!>1eWka=FE+~E4PxyBWQW zN%OW;-GA$s?vNovy_k15qumE383t$~nLwFAoPU+f`5lUy6LseP83o>|Dz+)g!i)Ht z3Xep;%i17jcKrFLa^7<2Zs2BEH{e5;oNtx+SBuk-qSG#323C-2j3u4%yVXQfTRRk= z(`@zUaNnFL9MMimVdJF5NPZFH_&M&_V?E^15#C~`Th<)Kr&=!w3N48Oz+wK!dai&& zTlte#{@yyz{Qy;Fq`wRRL0uYDLzfM49te){tUKXsywr!5#2*1Lcx@)shibfKewzJz zdqMsKjo1nRDA+Pd&A$G7Pn0;5W-Xu(`0)C^aNJzSqDUo&P#9gWwR#mCcGZKJ48amu zSkEfEVek{!ex59;nfr=nC`%VYbC+Ks@M|}w=>M9y3b&}&V7&`Jx=Rp{5;%0Xbc;07 zE8Qj1A>A$AB`gghjndsLAkqyBiqas>y_d&x?*0Sw%roCR`ObX#xHq%9?rVYWQ*-Ds zq?;Tsp{GZGAlE_6nS3zg0(lU^-U0N0Rqn2bBxQZ<$g*7pbTdD9YeR-657fz$-!c&^ z4&lU^D}1H9GjhYZZrRtFUN;jD1yMaQxs?k&iB+WTKXXC8(y77;G9JTF1|iU?CF-Hf z=q!9vv4k4+1&d(CrjK0|5tCRD+!9a-TB?Y|u6CX7Kc`>KW&vZhzJIuLx`mq9kvMah z;~409I&t#}=|mmMvXYTaemF^DBvFG9?j1_AcLK1UzdIqBWSDFB_-;!Nu@vjqcc8J9)vMPCI%q4vQ3qn;6h5hT6F*6G9m$+pSDGt-^l~ZXV51 zX^JLl9l_;fm_q8Lh-wfPy)|xN^AL~Y{!cqFX>ZWMXYEbUD?#oQwyEvhe@>8~wCMrp z5t%#COjK_4S+o{WhYDKnX1+Bu-~b_vN~^F#yJ%w^?*Gu5pytzfXjk5G%{RDbj2`_o z1OOP+rZ9O1^_~k!F5zB+j$%r`R&XciIP_U!$ETrxh~}(N(*Y2r@94=hBCo-mPrLBH zIqNZfS1SM{zv@ssG-2G$v)W$xDA>!}8>ggT8ZK2On>t+C5YO?5b|^JnumDW-Na2Gp z&S=&HNE2#Yt8)vH^r6*4{)KtYKfiJ+K@ry-i-hPZH9p6ueVpHFylxtjUieN2hE<~X zn3Q7yKZHTn-t&=Z6dARU>yl~+1p%#tvB*sHbiPPWQ})EGmRT50(DBh4Omg4c`t!8R zo6;jhP<$#XuI@w?TKz((@oL;}4{w8y1Zrz03Qu#g^aC(0C$V9viGZE%a8y)dtxQ zwqARwUZ6;62EBiUXz0%_y9`x1DHfm6c#?iEvBAxLImj2*`=nrDO{i%xVWY?mX}I{Z zqtmQw34V|vm}tAL<1#i1%VYb&x_kfW+-rAP;s;7E&-lcT>pY9Cmmq{$Rb6P&%X30v zV=aaodSq%f9u3YRaSHU({;%abA!SzM09kckqqfvE%h6_5e4cOaLmDqc0^t&2d5QU^L-`=1YV z@+ufH-MxEj=ZZDzW4**kbecr~5?~P%Z zeAfo{iX+Y9$$F>nM4Oq>Xk#UvIKw({Atpe@(_XUJq}7K&ZgLbfxcz{wNQR%q^dSd< z(#Cw}8pj&xp3OGt{mef;C7L7>@G3PaN%~hOMuL2G!zH;vE z>GlI?)*P{lyvLp)2oQ{)w1SL-4TRtpNo8DqVL14TxJzUmb$JS`DQPGsSu zwGa>j1J!IPRN@8~oZotSv$^8mF|mzyFGFt7z;7YAmLOtZXTmALQAIQB=n!3JV*4$r?+L(W``06qL zN7q}w;b9H?qa6r=d2e0%EFttUfLpOxpY>H>&!``{+kTvaNiR7JXk#zaU&#p?bQwyq zId~2+GZMh`d@vZi3Cfb(9Y>;bgE0x^1W(*=?Jl~jpVT<*rneroApQIsvmf>TqNTn= z#9N@04{!BrsKnJN<;Wqw*?M<~tr%qM#I7j%*#wHR}&<3oO_C0_gCnn&$N*N z4`AIbCckW+(_G4hTm&(XnsVqk%8jt(aGV3_0hSxrQ!$of z*5`X$t@q0@zIzEiH6iMX(ZyJ0?<}g^=Grjc)6VUe!-^ep4ol@^T}oxfWz^r=H}No89n4V$Ws<0ZKGF) zd8m<)Y}?A2bVZB?P*DKJ#*Jd=T2Zx!@FfptG6;cI?rgP6#(O|ZdaDGQs4Zi&7(ged zj`-E}UCIglJUZp*y(sNQEKx$!o|>jjV@u0+)b?-302pszS)V z5s@xc@)-z$u;>b?$;=d5X-7#ylXA<&j|9poTJ;Ah(__0UTYAVs01a))S>&9+bcXpQ zOOegg*Q@u<%cL{e!~|5cowrN=-=T8CNwCD9$M{8pM|p>OAOvo?Td)|F_vU?_%mS>? z%x>+M=%hGuZ;B%*JazyZHwljBYCehEanw9al5o*tNyXi{Hb@P#5>Xw)J94*R7VXFr z4IP#F^@Pn7halJ~z55s;Hed&ExfPRVS=^qMrNpLDMT+QmYFQb-guv z;JUk_6+LI)3etM*Ds+W+b$(Ew5}zTQfafv{ebZ;VNn#-?I*wN1?6t2TbWpPPgGdV) zMB{HGPF7!VV3hydWH$gw{N>*F|NqxQ?qfMvGB+e5?^*vzP#5mMDf;=l5^Jr@ zq;A|`tW@z@U?k|V>KzO*jR z*f0t`bn4yYHfHtIMGk|*aq|$<`k-;mM1SFq_>%w1jJqe)2^=!{yMN0uz%~NC(4Q1n zb%AwgiT^VTnPX>IP~_`CTGRcVSNE0FZ(|PtLtuNBEI)&i-C2ec|2sw{q0=i6!gk7# zY3caX2_AT)cR_j8Sd$BfPod1QUiw#H=OHS&T@nwQsu@C6-si zc}=+QkG02!$mNyo7D|UT0z>`tJ|WgP!^~>wW_5eJjH`ugrx&=5J6|#m$hla>rq7O& z(M_qtwAgDs4+cLLc-rL5hDY(4)6}puAFWL4xu0U$tg`yYb@}u7>&p|}>ZdRYp2t)0 zA_nLY?soyELD4AAN`=IO05(BjS-j*vSA+ua%Qoj*MuW5rZ`k|BE;8l@kPvRpc^gZG z=3>&)!f0tC8?MJ#Ia>V_2mbTnpce6_|@SaDh7EE8{KRl*$DMF z9~Ml@eq%6Gkt9&HJLe#yS3oSC&Vf;#z~>lls7qM5IjOn6UBz-AN`|M&>-4k^jUP?> zf($$Kl(z||<40fEED2&2GUmLp12^2Ew4+I$;G_-O9yJvgC-Wl-Cj1z|r2J+N`-;z+ zPGD0{X4q@OLPH~HAabA)Yf3o06c456wRHry(-l3wq@skItuAq71^DQ?7~)=z_?F|+ zA>n!|K14C04x7lZt7fJcD&EE6G zYQs;Tamp7dI^3MbIzHIAMegbPDhOecH8ITOuc{P(^IimZ!oIw_RbLw$t>cH=aVc4| zK#Sif&=`~5_gkm@Wqjo?P1I789gD5e)DLHFyB-${DYV0%$~~ICon)2}>nPIGEWZaK z%n2vOM{>fr-JqR9+>^B#iS_38LoH(Q!0SEz-po4rPn%VRgl>M`{pJRlKKoN(ivzv~ zrtau==3ujLc5)__ov+Uxgk-6y)euF(_~&vOo~F-7oV$)y1M;NSF+6v1ve-elQ(;Gp zVP5tIrQB1&Z_wfK`Dr&+gsBEHR#ag#JW6MCuz?p**tTghKB2Oai<}^Y-SjE>l4R-U z0(KPfEOvS4333O>Dd7cHv4EXvr1P%R-e-A|U2n3Omlb%klz`T*cW#2@xMO-;oyI8_ z#aK~jv>EbGoxzfSBlF`sO|v@av55`)NtbyDRFnyhR|QC_vGtMk2HjyhPH&#yd&n{V z?$Cpor%pRe=JG7B)Y^M&5t()U(H9DAmRrOZ0m=1X4JVfApSyI$ zhp6;yNf)f^0uzT$)c)i4EhcD$WkDcKpyH|syy#5+?_aNa?VM8 z#sZ=vvSx%V6lWvDjVamfol^|3yMq-OkPsV|?*lK+pLcZlnvEt78x_1|H!Ob8`l>BM zH{LHuJiPk{kXKj#1ud4^qld2*^SAP8pae$O)23X@D~j>L$C(!TJ0%mStmtNO!)LNA z!3q%4oyc(6G7!R~bgumC(-l0{4MF`|i-^&?O^(E(ns7+W{PSoxnF-PxBp}>)hRk=` zb{_NKn&3@)~0x273v2Qke|zWBA=}!-=(t zc0!s7L6TcwqB91g3k?Cj=~$hzsFXPLN_2`}0uk2>@LO{`qim|4F@5mnn6v54cQhiS ztQ_1xfAp6SRW+)rLm~;MUL4Qra1TtgD&rCM?y)-3XpUt&q;-L51bRi#?Q+SGr#g0~8hl@eisB!q1 z$Fp64_}}A`M|WKA7N~;zrBn3I5rlHLVPKnW&pzq(ml*T_*yyv^USkt7ZU2peJR`&52~!`!w~J24=v@yhbe7#g!FPQ zme{Lp8=|=Ck$++LIHg(j(PkrR#c*oMK-qMToe|I|`Q`DL^ zY0NYlUfnO&fyA)c=WYZgj-^yF&(qlf=2(#NR+5Y7L{DiwE}$ljlD7L!QMZ zziD*{IzHkk?vI`fBN!&hco{z9;`EA6?|vp@Stowyv0uCu%uuK!X+t* ztI(&=Y%~PMYH3j=txL1cwwIN9=WT~*1f#N%L*IM5aY$cU^#GS7$!M;s$11R&U#5d- z2fL=;d>kh7dz`iXM`vxk?oVJl-nDhkueu#Ysfk!YsFX+ zWgi-1hOrDG48z!Nnvi`fWXW3LHkOh-j6{~mxT$6&CdQUXl5MQln%y0@`}^Jcem?Jc zp3n1r&vVXsKJVu|@B7F597?5jHW-#7Ed9cBO6-zco7xAn%~;AE&jiCOS86rPJgQhb zk4C%uhS=`XaqCgAy|e_8Fp-A6`#BF~4(pvyAAX)QWAE3o6JSykb+uDN?gGB6FH%76 zT2N4w!_ZQ3gzH@{5P=4cKdtM>{yvyWk=cqWub3lRsI`gM^=#A5B`YE6H+=(jf;tMw zQ>RD7qdowXHoV_1_X+VHvxaFu=#V~t)S5`0Vc$()(FpNm@l1JAZVhMcYX{p4340hL%Yv%fO|0D66-V<*tgp|%bm;ro>eG&{+dbJ6Aml=kU!*{nG@B1XJG7^{SCMjBH>=_MN>Zv5ddlPIMl0nDorr(C& zpjc4s%E+Yl{*6MXxQr3NBkf0GizZdEdRRdXumRx+oq0`;j0(obKL}PTgcA5wCsMkumk9{2Fx$AK@&#mFhW>6vbpqWH(b89`&@?pTvyF$?qhZ5+I zLz6ih(E9F;LC;RM3kz&)uV42%9MjdcKz^JF4O=ev6WSl0dt4Hf2SX>%p0gDiZRByy z`e?w|!fDKC0~g`vQw{9>JC}>*26~06%6Oi18&~6iQf_8~_>BqjXYPEFg?sxzWx;*b z0W9FBD#Acb!0UYz1O~2$Lb@=E0xPBfJz?lP5~~C)ycb`?dxUpZxX1CzK%MGbJFm&?_+MdSpa$U0&n^`L>|2Q9!e8;H^x|(=$+$-4 zyNTSORkr!um+n^iV=6T^kprMEU|KK>*)CAqiHqV)Q2uZ zv{YwRx|8UVfPdC?zJ7;vb*iF#3gOOcGV(*KA?^gNm`@+y#B*pdpf4#PXWta6>hhVr zy_UEiadUQ!A1eR3&KH^HMXe;#!~L3d_Jf~9*m#N37n0Ie%~3~Hufcn|G4s!>2(7A( zlO2$aSoiN2!`+?KEtBv0R!(T9iW6PD=XGqsdl)&BiMhzvQ?nzC5ZSOY1A~}kr1FCI zHny|i0j+tYR>l7}ZnNoGTV?OeJZguSp@hdn9X`s1lqZxF5&5w^o36dsVdp9L&!434 z&4X~>ye>wk)#FN~hg5c-Tv$Hss>tJvMAw(j$(JtEL<+56 zh{firoP;00>Rm=C;!~So9O)cKe6w%Z(LUwAcTd^!q?GU|jX*7Xg&1|LKYc8tZUYvr zb89~|KB6r{LU)+>qG{?)RTU|#DSW=h!CLNM^ z@7F;V3gGwLdXb7$A2+Ia*+p&I$az%Bm3q7OJcc~7Ygt@$OK2i0`q|`A`}zg1#w;Z^ zk7!pF@v9nkPetBQlCutl%UG|TTYNOs z*fyiEDl1A!H?nkwC>w4Syr^$6sBkem<813Km?b$#oxAo~fJf`y=p8BjV;OcmN_>Bb z_e^~>Cqc4r{|<7+07HgC1-sJ-9HLoh(lvgyG$7UHi?LfG|l`g|E|yO zu>8IxmrQesrk8|)zos0x#6ALIm?;ZZl~%lXsZ+sXb7{;Xsr=QK^$rEvF)8CJp!3WK zi7wxNbxRo)?B=N#B;xr2f-mfSokuaNpe*u#-&&$B81%Mp zLZ-$EEqrUJqv0U-thCn5RDtE&Qb_zMO!ToJM?~TxC%?&%A5ZxAp?sA4Qgt3#zBU11#xlrG$ zjN(B*Z}= zT5r1&Seu^rj(v57bI}LlhVe=+zhp6A@ltlmAd&0r74#5T55vf5<2<#LPfz~-RhQ|O z2@tOy3XL|w8W^E42n#I6)(in;vp`?gN27Q>+&r)1VQ_0hED~XhMIlUqtjr$AkN#)3 zhNG+y|7VnWelYrI6EnC49D}w1!U(?ZFz4&eE*^N8hqEhza2@7>bN0i-koaJ0=d1pB zNv5Jt@V@Ts>wJUYd*p`$6x3#?6d{{RROg#M#&FS-WG#g*PB$t)bQJKTl;w80O^Og&wCFncCu`2YayfEZI_ z`Iw&eIJ)2Xa7z6&O+2mO)PoHri2?P&j9;At^=+6pAEam|fcoXkR)bZK8L)p1yUZ}g z{~E`Oa18thYm~#3qQ9~SiTGcZ!j!r!=9e%Q2k#FjC4ZQiC%qm3X#9!Uvi#7IYD~#| RYM20G7ny;XKF|)XzX3^uE9L+I literal 0 HcmV?d00001 -- 1.7.4.4