HBASE-15477 Purge 'next block header' from cached blocks

When we read from HDFS, we overread to pick up the next blocks header.
Doing this saves a seek as we move through the hfile; we save having to
do an explicit seek just to read the block header every time we need to
read the body.  We used to read in the next header as part of the
current blocks buffer. This buffer was then what got persisted to
blockcache; so we were over-persisting: our block plus the next blocks'
header (33 bytes).

This patch undoes this over-persisting.

Removes support for version 1 blocks (0.2 was added in hbase-0.92.0).
Not needed any more.

There is an open question on whether checksums should be persisted
when caching. The code seems to say no but if cache is SSD backed or
backed by anything that does not do error correction, we'll want
checksums.

Adds loads of documentation.

M hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
  (write) Add writing from a ByteBuff.

M hbase-common/src/main/java/org/apache/hadoop/hbase/nio/ByteBuff.java
  (toString) Add one so ByteBuff looks like ByteBuffer when you click on
  it in IDE

M hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
  Remove support for version 1 blocks.

  Cleaned up handling of metadata added when we serialize a block to
  caches. Metadata is smaller now.

  When we serialize (used when caching), do not persist the next blocks
  header if present.

  Removed a bunch of methods, a few of which had overlapping
  functionality and others that exposed too much of our internals.
  Also removed a bunch of constructors and unified the constructors we
  had left over making them share a common init method.
  Shutdown access to defines that should only be used internally here.

  Renamed all to do w/ 'EXTRA' and 'extraSerialization' to instead talk
  about metadata saved to caches; was unclear previously what EXTRA was
  about.

  Renamed static final declarations as all uppercase.

  (readBlockDataInternal): Redid. Couldn't make sense of it previously.
  Undid heavy-duty parse of header by constructing HFileBlock. Other
  cleanups. Its 1/3rd the length it used to be. More to do in here.

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
stack 2016-04-21 22:38:51 -07:00
parent 4272178b1f
commit 6b78409eb2
26 changed files with 632 additions and 2439 deletions

View File

@ -56,6 +56,7 @@ public class HFileContext implements HeapSize, Cloneable {
/** Encryption algorithm and key used */
private Encryption.Context cryptoContext = Encryption.Context.NONE;
private long fileCreateTime;
private String hfileName;
//Empty constructor. Go with setters
public HFileContext() {
@ -77,12 +78,13 @@ public class HFileContext implements HeapSize, Cloneable {
this.encoding = context.encoding;
this.cryptoContext = context.cryptoContext;
this.fileCreateTime = context.fileCreateTime;
this.hfileName = context.hfileName;
}
public HFileContext(boolean useHBaseChecksum, boolean includesMvcc, boolean includesTags,
Compression.Algorithm compressAlgo, boolean compressTags, ChecksumType checksumType,
int bytesPerChecksum, int blockSize, DataBlockEncoding encoding,
Encryption.Context cryptoContext, long fileCreateTime) {
Encryption.Context cryptoContext, long fileCreateTime, String hfileName) {
this.usesHBaseChecksum = useHBaseChecksum;
this.includesMvcc = includesMvcc;
this.includesTags = includesTags;
@ -96,6 +98,7 @@ public class HFileContext implements HeapSize, Cloneable {
}
this.cryptoContext = cryptoContext;
this.fileCreateTime = fileCreateTime;
this.hfileName = hfileName;
}
/**
@ -187,6 +190,10 @@ public class HFileContext implements HeapSize, Cloneable {
this.cryptoContext = cryptoContext;
}
public String getHFileName() {
return this.hfileName;
}
/**
* HeapSize implementation
* NOTE : The heapsize should be altered as and when new state variable are added
@ -196,7 +203,7 @@ public class HFileContext implements HeapSize, Cloneable {
public long heapSize() {
long size = ClassSize.align(ClassSize.OBJECT +
// Algorithm reference, encodingon, checksumtype, Encryption.Context reference
4 * ClassSize.REFERENCE +
5 * ClassSize.REFERENCE +
2 * Bytes.SIZEOF_INT +
// usesHBaseChecksum, includesMvcc, includesTags and compressTags
4 * Bytes.SIZEOF_BOOLEAN +

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.util.ChecksumType;
/**
* A builder that helps in building up the HFileContext
* A builder that helps in building up the HFileContext
*/
@InterfaceAudience.Private
public class HFileContextBuilder {
@ -53,6 +53,28 @@ public class HFileContextBuilder {
private Encryption.Context cryptoContext = Encryption.Context.NONE;
private long fileCreateTime = 0;
private String hfileName = null;
public HFileContextBuilder() {}
/**
* Use this constructor if you want to change a few settings only in another context.
*/
public HFileContextBuilder(final HFileContext hfc) {
this.usesHBaseChecksum = hfc.isUseHBaseChecksum();
this.includesMvcc = hfc.isIncludesMvcc();
this.includesTags = hfc.isIncludesTags();
this.compression = hfc.getCompression();
this.compressTags = hfc.isCompressTags();
this.checksumType = hfc.getChecksumType();
this.bytesPerChecksum = hfc.getBytesPerChecksum();
this.blocksize = hfc.getBlocksize();
this.encoding = hfc.getDataBlockEncoding();
this.cryptoContext = hfc.getEncryptionContext();
this.fileCreateTime = hfc.getFileCreateTime();
this.hfileName = hfc.getHFileName();
}
public HFileContextBuilder withHBaseCheckSum(boolean useHBaseCheckSum) {
this.usesHBaseChecksum = useHBaseCheckSum;
return this;
@ -108,9 +130,14 @@ public class HFileContextBuilder {
return this;
}
public HFileContextBuilder withHFileName(String name) {
this.hfileName = name;
return this;
}
public HFileContext build() {
return new HFileContext(usesHBaseChecksum, includesMvcc, includesTags, compression,
compressTags, checksumType, bytesPerChecksum, blocksize, encoding, cryptoContext,
fileCreateTime);
fileCreateTime, hfileName);
}
}

View File

@ -255,7 +255,7 @@ public class MemcachedBlockCache implements BlockCache {
public HFileBlock decode(CachedData d) {
try {
ByteBuffer buf = ByteBuffer.wrap(d.getData());
return (HFileBlock) HFileBlock.blockDeserializer.deserialize(buf, true);
return (HFileBlock) HFileBlock.BLOCK_DESERIALIZER.deserialize(buf, true);
} catch (IOException e) {
LOG.warn("Error deserializing data from memcached",e);
}

View File

@ -91,7 +91,7 @@ public class ChecksumUtil {
// If this is an older version of the block that does not have
// checksums, then return false indicating that checksum verification
// did not succeed. Actually, this methiod should never be called
// did not succeed. Actually, this method should never be called
// when the minorVersion is 0, thus this is a defensive check for a
// cannot-happen case. Since this is a cannot-happen case, it is
// better to return false to indicate a checksum validation failure.
@ -141,8 +141,7 @@ public class ChecksumUtil {
* @return The number of bytes needed to store the checksum values
*/
static long numBytes(long datasize, int bytesPerChecksum) {
return numChunks(datasize, bytesPerChecksum) *
HFileBlock.CHECKSUM_SIZE;
return numChunks(datasize, bytesPerChecksum) * HFileBlock.CHECKSUM_SIZE;
}
/**

View File

@ -201,10 +201,11 @@ public class HFileReaderV2 extends AbstractHFileReader {
if (Thread.interrupted()) {
break;
}
long onDiskSize = -1;
if (prevBlock != null) {
onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
}
// Perhaps we got our block from cache? Unlikely as this may be, if it happens, then
// the internal-to-hfileblock thread local which holds the overread that gets the
// next header, will not have happened...so, pass in the onDiskSize gotten from the
// cached block. This 'optimization' triggers extremely rarely I'd say.
long onDiskSize = prevBlock != null? prevBlock.getNextBlockOnDiskSize(): -1;
HFileBlock block = readBlock(offset, onDiskSize, true, false, false, false,
null, null);
prevBlock = block;
@ -370,7 +371,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset,
blockSize, -1, true).unpack(hfileContext, fsBlockReader);
blockSize, true).unpack(hfileContext, fsBlockReader);
// Cache the block
if (cacheBlock) {
@ -450,7 +451,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
traceScope.getSpan().addTimelineAnnotation("blockCacheMiss");
}
// Load block from filesystem.
HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, -1,
HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize,
pread);
validateBlockType(hfileBlock, expectedBlockType);
HFileBlock unpacked = hfileBlock.unpack(hfileContext, fsBlockReader);
@ -729,7 +730,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
// it might turn out to be a non-data block.
curBlock = reader.readBlock(curBlock.getOffset()
+ curBlock.getOnDiskSizeWithHeader(),
curBlock.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread,
curBlock.getNextBlockOnDiskSize(), cacheBlocks, pread,
isCompaction, true, null, getEffectiveDataBlockEncoding());
} while (!curBlock.getBlockType().isData());

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.nio.BufferUnderflowException;
import java.security.Key;
import java.security.KeyException;
@ -277,11 +278,15 @@ public class HFileReaderV3 extends HFileReaderV2 {
int lastKeyValueSize = -1;
KeyValue.KeyOnlyKeyValue keyOnlyKv = new KeyValue.KeyOnlyKeyValue();
do {
blockBuffer.mark();
klen = blockBuffer.getInt();
vlen = blockBuffer.getInt();
if (klen < 0 || vlen < 0 || klen > blockBuffer.limit()
|| vlen > blockBuffer.limit()) {
try {
blockBuffer.mark();
klen = blockBuffer.getInt();
vlen = blockBuffer.getInt();
} catch (BufferUnderflowException bufe) {
LOG.error("this.blockBuffer=" + this.blockBuffer, bufe);
throw bufe;
}
if (klen < 0 || vlen < 0 || klen > blockBuffer.limit() || vlen > blockBuffer.limit()) {
throw new IllegalStateException("Invalid klen " + klen + " or vlen "
+ vlen + ". Block offset: "
+ block.getOffset() + ", block length: " + blockBuffer.limit() + ", position: "

View File

@ -1245,25 +1245,22 @@ public class BucketCache implements BlockCache, HeapSize {
final AtomicLong realCacheSize) throws CacheFullException, IOException,
BucketAllocatorException {
int len = data.getSerializedLength();
// This cacheable thing can't be serialized...
// This cacheable thing can't be serialized
if (len == 0) return null;
long offset = bucketAllocator.allocateBlock(len);
BucketEntry bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
try {
if (data instanceof HFileBlock) {
HFileBlock block = (HFileBlock) data;
ByteBuffer sliceBuf = block.getBufferReadOnlyWithHeader();
sliceBuf.rewind();
assert len == sliceBuf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE ||
len == sliceBuf.limit() + block.headerSize() + HFileBlock.EXTRA_SERIALIZATION_SPACE;
ByteBuffer extraInfoBuffer = ByteBuffer.allocate(HFileBlock.EXTRA_SERIALIZATION_SPACE);
block.serializeExtraInfo(extraInfoBuffer);
// If an instance of HFileBlock, save on some allocations.
HFileBlock block = (HFileBlock)data;
ByteBuffer sliceBuf = block.getBufferReadOnly();
ByteBuffer metadata = block.getMetaData();
if (LOG.isTraceEnabled()) {
LOG.trace("Write offset=" + offset + ", len=" + len);
}
ioEngine.write(sliceBuf, offset);
ioEngine.write(extraInfoBuffer, offset + len - HFileBlock.EXTRA_SERIALIZATION_SPACE);
ioEngine.write(metadata, offset + len - metadata.limit());
} else {
ByteBuffer bb = ByteBuffer.allocate(len);
data.serialize(bb);

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.Closeable;
import java.io.IOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -74,6 +75,7 @@ public interface KeyValueScanner {
* The default implementation for this would be to return 0. A file having
* lower sequence id will be considered to be the older one.
*/
// TODO: Implement SequenceId Interface instead.
long getSequenceID();
/**

View File

@ -18,12 +18,6 @@
*/
package org.apache.hadoop.hbase.regionserver;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Ordering;
import java.io.DataInput;
import java.io.IOException;
import java.net.InetSocketAddress;
@ -65,9 +59,14 @@ import org.apache.hadoop.hbase.util.BloomFilter;
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.Writables;
import org.apache.hadoop.io.WritableUtils;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
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
@ -1212,7 +1211,7 @@ public class StoreFile {
}
/**
* Warning: Do not write further code which depends on this call. Instead
* @deprecated Do not write further code which depends on this call. Instead
* use getStoreFileScanner() which uses the StoreFileScanner class/interface
* which is the preferred way to scan a store with higher level concepts.
*
@ -1226,7 +1225,7 @@ public class StoreFile {
}
/**
* Warning: Do not write further code which depends on this call. Instead
* @deprecated Do not write further code which depends on this call. Instead
* use getStoreFileScanner() which uses the StoreFileScanner class/interface
* which is the preferred way to scan a store with higher level concepts.
*

View File

@ -1,433 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import junit.framework.Assert;
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.FileUtil;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.migration.NamespaceUpgrade;
import org.apache.hadoop.hbase.client.HTable;
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.client.Durability;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.util.ToolRunner;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test migration that changes HRI serialization into PB. Tests by bringing up a cluster from actual
* data from a 0.92 cluster, as well as manually downgrading and then upgrading the hbase:meta info.
* @deprecated Remove after 0.96
*/
@Category(MediumTests.class)
@Deprecated
public class TestMetaMigrationConvertingToPB {
static final Log LOG = LogFactory.getLog(TestMetaMigrationConvertingToPB.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final static String TESTTABLE = "TestTable";
private final static int ROW_COUNT = 100;
private final static int REGION_COUNT = 9; //initial number of regions of the TestTable
private static final int META_VERSION_092 = 0;
/*
* This test uses a tgz file named "TestMetaMigrationConvertingToPB.tgz" under
* hbase-server/src/test/data which contains file data from a 0.92 cluster.
* The cluster has a table named "TestTable", which has 100 rows. 0.94 has same
* hbase:meta structure, so it should be the same.
*
* hbase(main):001:0> create 'TestTable', 'f1'
* hbase(main):002:0> for i in 1..100
* hbase(main):003:1> put 'TestTable', "row#{i}", "f1:c1", i
* hbase(main):004:1> end
*
* There are 9 regions in the table
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Start up our mini cluster on top of an 0.92 root.dir that has data from
// a 0.92 hbase run -- it has a table with 100 rows in it -- and see if
// we can migrate from 0.92
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.startMiniDFSCluster(1);
Path testdir = TEST_UTIL.getDataTestDir("TestMetaMigrationConvertToPB");
// Untar our test dir.
File untar = untar(new File(testdir.toString()));
// Now copy the untar up into hdfs so when we start hbase, we'll run from it.
Configuration conf = TEST_UTIL.getConfiguration();
FsShell shell = new FsShell(conf);
FileSystem fs = FileSystem.get(conf);
// find where hbase will root itself, so we can copy filesystem there
Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
if (!fs.isDirectory(hbaseRootDir.getParent())) {
// mkdir at first
fs.mkdirs(hbaseRootDir.getParent());
}
doFsCommand(shell,
new String [] {"-put", untar.toURI().toString(), hbaseRootDir.toString()});
// windows fix: tgz file has hbase:meta directory renamed as -META- since the original
// is an illegal name under windows. So we rename it back.
// See src/test/data//TestMetaMigrationConvertingToPB.README and
// https://issues.apache.org/jira/browse/HBASE-6821
doFsCommand(shell, new String [] {"-mv", new Path(hbaseRootDir, "-META-").toString(),
new Path(hbaseRootDir, ".META.").toString()});
// See whats in minihdfs.
doFsCommand(shell, new String [] {"-lsr", "/"});
//upgrade to namespace as well
Configuration toolConf = TEST_UTIL.getConfiguration();
conf.set(HConstants.HBASE_DIR, TEST_UTIL.getDefaultRootDirPath().toString());
ToolRunner.run(toolConf, new NamespaceUpgrade(), new String[]{"--upgrade"});
TEST_UTIL.startMiniHBaseCluster(1, 1);
// Assert we are running against the copied-up filesystem. The copied-up
// rootdir should have had a table named 'TestTable' in it. Assert it
// present.
HTable t = new HTable(TEST_UTIL.getConfiguration(), TESTTABLE);
ResultScanner scanner = t.getScanner(new Scan());
int count = 0;
while (scanner.next() != null) {
count++;
}
// Assert that we find all 100 rows that are in the data we loaded. If
// so then we must have migrated it from 0.90 to 0.92.
Assert.assertEquals(ROW_COUNT, count);
scanner.close();
t.close();
}
private static File untar(final File testdir) throws IOException {
// Find the src data under src/test/data
final String datafile = "TestMetaMigrationConvertToPB";
String srcTarFile =
System.getProperty("project.build.testSourceDirectory", "src/test") +
File.separator + "data" + File.separator + datafile + ".tgz";
File homedir = new File(testdir.toString());
File tgtUntarDir = new File(homedir, datafile);
if (tgtUntarDir.exists()) {
if (!FileUtil.fullyDelete(tgtUntarDir)) {
throw new IOException("Failed delete of " + tgtUntarDir.toString());
}
}
LOG.info("Untarring " + srcTarFile + " into " + homedir.toString());
FileUtil.unTar(new File(srcTarFile), homedir);
Assert.assertTrue(tgtUntarDir.exists());
return tgtUntarDir;
}
private static void doFsCommand(final FsShell shell, final String [] args)
throws Exception {
// Run the 'put' command.
int errcode = shell.run(args);
if (errcode != 0) throw new IOException("Failed put; errcode=" + errcode);
}
/**
* @throws java.lang.Exception
*/
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testMetaUpdatedFlagInROOT() throws Exception {
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
boolean metaUpdated = MetaMigrationConvertingToPB.
isMetaTableUpdated(master.getConnection());
assertEquals(true, metaUpdated);
verifyMetaRowsAreUpdated(master.getConnection());
}
@Test
public void testMetaMigration() throws Exception {
LOG.info("Starting testMetaMigration");
final byte [] FAMILY = Bytes.toBytes("family");
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testMetaMigration"));
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
htd.addFamily(hcd);
Configuration conf = TEST_UTIL.getConfiguration();
byte[][] regionNames = new byte[][]{
HConstants.EMPTY_START_ROW,
Bytes.toBytes("region_a"),
Bytes.toBytes("region_b")};
createMultiRegionsWithWritableSerialization(conf,
htd.getTableName().getName(),
regionNames);
HConnection masterHConnection =
TEST_UTIL.getMiniHBaseCluster().getMaster().getConnection();
// Erase the current version of root meta for this test.
undoVersionInRoot();
MetaTableAccessor.fullScanMetaAndPrint(masterHConnection);
LOG.info("Meta Print completed.testMetaMigration");
long numMigratedRows = MetaMigrationConvertingToPB.updateMeta(
TEST_UTIL.getHBaseCluster().getMaster());
MetaTableAccessor.fullScanMetaAndPrint(masterHConnection);
// Should be one entry only and it should be for the table we just added.
assertEquals(regionNames.length, numMigratedRows);
// Assert that the flag in ROOT is updated to reflect the correct status
boolean metaUpdated = MetaMigrationConvertingToPB.isMetaTableUpdated(masterHConnection);
assertEquals(true, metaUpdated);
verifyMetaRowsAreUpdated(masterHConnection);
}
/**
* This test assumes a master crash/failure during the meta migration process
* and attempts to continue the meta migration process when a new master takes over.
* When a master dies during the meta migration we will have some rows of
* META.CatalogFamily updated with PB serialization and some
* still hanging with writable serialization. When the backup master/ or
* fresh start of master attempts the migration it will encounter some rows of META
* already updated with new HRI and some still legacy. This test will simulate this
* scenario and validates that the migration process can safely skip the updated
* rows and migrate any pending rows at startup.
* @throws Exception
*/
@Test
public void testMasterCrashDuringMetaMigration() throws Exception {
final byte[] FAMILY = Bytes.toBytes("family");
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf
("testMasterCrashDuringMetaMigration"));
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
htd.addFamily(hcd);
Configuration conf = TEST_UTIL.getConfiguration();
// Create 10 New regions.
createMultiRegionsWithPBSerialization(conf, htd.getTableName().getName(), 10);
// Create 10 Legacy regions.
createMultiRegionsWithWritableSerialization(conf,
htd.getTableName().getName(), 10);
HConnection masterHConnection =
TEST_UTIL.getMiniHBaseCluster().getMaster().getConnection();
// Erase the current version of root meta for this test.
undoVersionInRoot();
MetaTableAccessor.fullScanMetaAndPrint(masterHConnection);
LOG.info("Meta Print completed.testUpdatesOnMetaWithLegacyHRI");
long numMigratedRows =
MetaMigrationConvertingToPB.updateMetaIfNecessary(
TEST_UTIL.getHBaseCluster().getMaster());
assertEquals(numMigratedRows, 10);
// Assert that the flag in ROOT is updated to reflect the correct status
boolean metaUpdated = MetaMigrationConvertingToPB.isMetaTableUpdated(masterHConnection);
assertEquals(true, metaUpdated);
verifyMetaRowsAreUpdated(masterHConnection);
LOG.info("END testMasterCrashDuringMetaMigration");
}
/**
* Verify that every hbase:meta row is updated
*/
void verifyMetaRowsAreUpdated(HConnection hConnection)
throws IOException {
List<Result> results = MetaTableAccessor.fullScan(hConnection);
assertTrue(results.size() >= REGION_COUNT);
for (Result result : results) {
byte[] hriBytes = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
assertTrue(hriBytes != null && hriBytes.length > 0);
assertTrue(MetaMigrationConvertingToPB.isMigrated(hriBytes));
byte[] splitA = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER);
if (splitA != null && splitA.length > 0) {
assertTrue(MetaMigrationConvertingToPB.isMigrated(splitA));
}
byte[] splitB = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER);
if (splitB != null && splitB.length > 0) {
assertTrue(MetaMigrationConvertingToPB.isMigrated(splitB));
}
}
}
/** Changes the version of hbase:meta to 0 to simulate 0.92 and 0.94 clusters*/
private void undoVersionInRoot() throws IOException {
Put p = new Put(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
p.add(HConstants.CATALOG_FAMILY, HConstants.META_VERSION_QUALIFIER,
Bytes.toBytes(META_VERSION_092));
// TODO wire this MetaEditor.putToRootTable(ct, p);
LOG.info("Downgraded -ROOT- meta version=" + META_VERSION_092);
}
/**
* Inserts multiple regions into hbase:meta using Writable serialization instead of PB
*/
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final byte[] tableName, int numRegions) throws IOException {
if (numRegions < 3) throw new IOException("Must create at least 3 regions");
byte [] startKey = Bytes.toBytes("aaaaa");
byte [] endKey = Bytes.toBytes("zzzzz");
byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
byte [][] regionStartKeys = new byte[splitKeys.length+1][];
for (int i=0;i<splitKeys.length;i++) {
regionStartKeys[i+1] = splitKeys[i];
}
regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
return createMultiRegionsWithWritableSerialization(c, tableName, regionStartKeys);
}
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final byte[] tableName, byte [][] startKeys)
throws IOException {
return createMultiRegionsWithWritableSerialization(c,
TableName.valueOf(tableName), startKeys);
}
/**
* Inserts multiple regions into hbase:meta using Writable serialization instead of PB
*/
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final TableName tableName, byte [][] startKeys)
throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, TableName.META_TABLE_NAME);
List<HRegionInfo> newRegions
= new ArrayList<HRegionInfo>(startKeys.length);
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(tableName, startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
put.setDurability(Durability.SKIP_WAL);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
getBytes(hri)); //this is the old Writable serialization
//also add the region as it's daughters
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
getBytes(hri)); //this is the old Writable serialization
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
getBytes(hri)); //this is the old Writable serialization
meta.put(put);
LOG.info("createMultiRegionsWithWritableSerialization: PUT inserted " + hri.toString());
newRegions.add(hri);
count++;
}
meta.close();
return count;
}
@Deprecated
private byte[] getBytes(HRegionInfo hri) throws IOException {
DataOutputBuffer out = new DataOutputBuffer();
try {
hri.write(out);
return out.getData();
} finally {
if (out != null) {
out.close();
}
}
}
/**
* Inserts multiple regions into hbase:meta using PB serialization
*/
int createMultiRegionsWithPBSerialization(final Configuration c,
final byte[] tableName, int numRegions)
throws IOException {
if (numRegions < 3) throw new IOException("Must create at least 3 regions");
byte [] startKey = Bytes.toBytes("aaaaa");
byte [] endKey = Bytes.toBytes("zzzzz");
byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
byte [][] regionStartKeys = new byte[splitKeys.length+1][];
for (int i=0;i<splitKeys.length;i++) {
regionStartKeys[i+1] = splitKeys[i];
}
regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
return createMultiRegionsWithPBSerialization(c, tableName, regionStartKeys);
}
/**
* Inserts multiple regions into hbase:meta using PB serialization
*/
int createMultiRegionsWithPBSerialization(final Configuration c, final byte[] tableName,
byte [][] startKeys) throws IOException {
return createMultiRegionsWithPBSerialization(c,
TableName.valueOf(tableName), startKeys);
}
int createMultiRegionsWithPBSerialization(final Configuration c,
final TableName tableName,
byte [][] startKeys) throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, TableName.META_TABLE_NAME);
List<HRegionInfo> newRegions
= new ArrayList<HRegionInfo>(startKeys.length);
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(tableName, startKeys[i], startKeys[j]);
Put put = MetaTableAccessor.makePutFromRegionInfo(hri);
put.setDurability(Durability.SKIP_WAL);
meta.put(put);
LOG.info("createMultiRegionsWithPBSerialization: PUT inserted " + hri.toString());
newRegions.add(hri);
count++;
}
meta.close();
return count;
}
}

View File

@ -64,6 +64,7 @@ public class CacheTestUtils {
/*Post eviction, heapsize should be the same */
assertEquals(heapSize, ((HeapSize) toBeTested).heapSize());
}
public static void testCacheMultiThreaded(final BlockCache toBeTested,
final int blockSize, final int numThreads, final int numQueries,
final double passingScore) throws Exception {
@ -332,25 +333,16 @@ public class CacheTestUtils {
}
private static HFileBlockPair[] generateHFileBlocks(int blockSize,
int numBlocks) {
private static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks) {
HFileBlockPair[] returnedBlocks = new HFileBlockPair[numBlocks];
Random rand = new Random();
HashSet<String> usedStrings = new HashSet<String>();
for (int i = 0; i < numBlocks; i++) {
// The buffer serialized size needs to match the size of BlockSize. So we
// declare our data size to be smaller than it by the serialization space
// required.
ByteBuffer cachedBuffer = ByteBuffer.allocate(blockSize
- HFileBlock.EXTRA_SERIALIZATION_SPACE);
ByteBuffer cachedBuffer = ByteBuffer.allocate(blockSize);
rand.nextBytes(cachedBuffer.array());
cachedBuffer.rewind();
int onDiskSizeWithoutHeader = blockSize
- HFileBlock.EXTRA_SERIALIZATION_SPACE;
int uncompressedSizeWithoutHeader = blockSize
- HFileBlock.EXTRA_SERIALIZATION_SPACE;
int onDiskSizeWithoutHeader = blockSize;
int uncompressedSizeWithoutHeader = blockSize;
long prevBlockOffset = rand.nextLong();
BlockType.DATA.write(cachedBuffer);
cachedBuffer.putInt(onDiskSizeWithoutHeader);
@ -369,7 +361,7 @@ public class CacheTestUtils {
onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
prevBlockOffset, cachedBuffer, HFileBlock.DONT_FILL_HEADER,
blockSize,
onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, meta);
onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, -1, meta);
String strKey;
/* No conflicting keys */

View File

@ -259,20 +259,15 @@ public class TestCacheOnWrite {
assertTrue(testDescription, scanner.seekTo());
long offset = 0;
HFileBlock prevBlock = null;
EnumMap<BlockType, Integer> blockCountByType =
new EnumMap<BlockType, Integer>(BlockType.class);
DataBlockEncoding encodingInCache = NoOpDataBlockEncoder.INSTANCE.getDataBlockEncoding();
while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
long onDiskSize = -1;
if (prevBlock != null) {
onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
}
// Flags: don't cache the block, use pread, this is not a compaction.
// Also, pass null for expected block type to avoid checking it.
HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
false, true, null, encodingInCache);
HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null,
encodingInCache);
BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
offset);
HFileBlock fromCache = (HFileBlock) blockCache.getBlock(blockCacheKey, true, false, true);
@ -303,7 +298,6 @@ public class TestCacheOnWrite {
assertEquals(
block.getUncompressedSizeWithoutHeader(), fromCache.getUncompressedSizeWithoutHeader());
}
prevBlock = block;
offset += block.getOnDiskSizeWithHeader();
BlockType bt = block.getBlockType();
Integer count = blockCountByType.get(bt);
@ -399,7 +393,7 @@ public class TestCacheOnWrite {
final String cf = "myCF";
final byte[] cfBytes = Bytes.toBytes(cf);
final int maxVersions = 3;
Region region = TEST_UTIL.createTestRegion(table,
Region region = TEST_UTIL.createTestRegion(table,
new HColumnDescriptor(cf)
.setCompressionType(compress)
.setBloomFilterType(BLOOM_TYPE)
@ -410,7 +404,7 @@ public class TestCacheOnWrite {
long ts = EnvironmentEdgeManager.currentTime();
for (int iFile = 0; iFile < 5; ++iFile) {
for (int iRow = 0; iRow < 500; ++iRow) {
String rowStr = "" + (rowIdx * rowIdx * rowIdx) + "row" + iFile + "_" +
String rowStr = "" + (rowIdx * rowIdx * rowIdx) + "row" + iFile + "_" +
iRow;
Put p = new Put(Bytes.toBytes(rowStr));
++rowIdx;

View File

@ -93,7 +93,7 @@ public class TestChecksum {
meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
is, totalSize, (HFileSystem) fs, path, meta);
HFileBlock b = hbr.readBlockData(0, -1, -1, false);
HFileBlock b = hbr.readBlockData(0, -1, false);
assertEquals(b.getChecksumType(), ChecksumType.getDefaultChecksumType().getCode());
}
@ -107,12 +107,14 @@ public class TestChecksum {
ChecksumType cktype = itr.next();
Path path = new Path(TEST_UTIL.getDataTestDir(), "checksum" + cktype.getName());
FSDataOutputStream os = fs.create(path);
HFileContext meta = new HFileContextBuilder()
.withChecksumType(cktype).build();
HFileContext meta = new HFileContextBuilder().
withChecksumType(cktype).
build();
HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
DataOutputStream dos = hbw.startWriting(BlockType.DATA);
for (int i = 0; i < 1000; ++i)
for (int i = 0; i < 1000; ++i) {
dos.writeInt(i);
}
hbw.writeHeaderAndData(os);
int totalSize = hbw.getOnDiskSizeWithHeader();
os.close();
@ -124,7 +126,7 @@ public class TestChecksum {
meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
is, totalSize, (HFileSystem) fs, path, meta);
HFileBlock b = hbr.readBlockData(0, -1, -1, false);
HFileBlock b = hbr.readBlockData(0, -1, false);
ByteBuffer data = b.getBufferWithoutHeader();
for (int i = 0; i < 1000; i++) {
assertEquals(i, data.getInt());
@ -175,7 +177,7 @@ public class TestChecksum {
}
os.close();
// Use hbase checksums.
// Use hbase checksums.
assertEquals(true, hfs.useHBaseChecksum());
// Do a read that purposely introduces checksum verification failures.
@ -187,10 +189,10 @@ public class TestChecksum {
.withHBaseCheckSum(true)
.build();
HFileBlock.FSReader hbr = new FSReaderImplTest(is, totalSize, fs, path, meta);
HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
HFileBlock b = hbr.readBlockData(0, -1, pread);
b.sanityCheck();
assertEquals(4936, b.getUncompressedSizeWithoutHeader());
assertEquals(algo == GZ ? 2173 : 4936,
assertEquals(algo == GZ ? 2173 : 4936,
b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
// read data back from the hfile, exclude header and checksum
ByteBuffer bb = b.unpack(meta, hbr).getBufferWithoutHeader(); // read back data
@ -206,35 +208,35 @@ public class TestChecksum {
// A single instance of hbase checksum failure causes the reader to
// switch off hbase checksum verification for the next 100 read
// requests. Verify that this is correct.
for (int i = 0; i <
for (int i = 0; i <
HFileBlock.CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD + 1; i++) {
b = hbr.readBlockData(0, -1, -1, pread);
b = hbr.readBlockData(0, -1, pread);
assertEquals(0, HFile.getChecksumFailuresCount());
}
// The next read should have hbase checksum verification reanabled,
// we verify this by assertng that there was a hbase-checksum failure.
b = hbr.readBlockData(0, -1, -1, pread);
b = hbr.readBlockData(0, -1, pread);
assertEquals(1, HFile.getChecksumFailuresCount());
// Since the above encountered a checksum failure, we switch
// back to not checking hbase checksums.
b = hbr.readBlockData(0, -1, -1, pread);
b = hbr.readBlockData(0, -1, pread);
assertEquals(0, HFile.getChecksumFailuresCount());
is.close();
// Now, use a completely new reader. Switch off hbase checksums in
// Now, use a completely new reader. Switch off hbase checksums in
// the configuration. In this case, we should not detect
// any retries within hbase.
// any retries within hbase.
HFileSystem newfs = new HFileSystem(TEST_UTIL.getConfiguration(), false);
assertEquals(false, newfs.useHBaseChecksum());
is = new FSDataInputStreamWrapper(newfs, path);
hbr = new FSReaderImplTest(is, totalSize, newfs, path, meta);
b = hbr.readBlockData(0, -1, -1, pread);
b = hbr.readBlockData(0, -1, pread);
is.close();
b.sanityCheck();
b = b.unpack(meta, hbr);
assertEquals(4936, b.getUncompressedSizeWithoutHeader());
assertEquals(algo == GZ ? 2173 : 4936,
assertEquals(algo == GZ ? 2173 : 4936,
b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
// read data back from the hfile, exclude header and checksum
bb = b.getBufferWithoutHeader(); // read back data
@ -249,7 +251,7 @@ public class TestChecksum {
}
}
/**
/**
* Test different values of bytesPerChecksum
*/
@Test
@ -262,7 +264,7 @@ public class TestChecksum {
Compression.Algorithm algo = NONE;
for (boolean pread : new boolean[] { false, true }) {
for (int bytesPerChecksum : BYTES_PER_CHECKSUM) {
Path path = new Path(TEST_UTIL.getDataTestDir(), "checksumChunk_" +
Path path = new Path(TEST_UTIL.getDataTestDir(), "checksumChunk_" +
algo + bytesPerChecksum);
FSDataOutputStream os = fs.create(path);
HFileContext meta = new HFileContextBuilder()
@ -298,7 +300,7 @@ public class TestChecksum {
", dataSize=" + dataSize +
", expectedChunks=" + expectedChunks);
// Verify hbase checksums.
// Verify hbase checksums.
assertEquals(true, hfs.useHBaseChecksum());
// Read data back from file.
@ -313,7 +315,7 @@ public class TestChecksum {
.build();
HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(
is, nochecksum), totalSize, hfs, path, meta);
HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
HFileBlock b = hbr.readBlockData(0, -1, pread);
is.close();
b.sanityCheck();
assertEquals(dataSize, b.getUncompressedSizeWithoutHeader());
@ -337,7 +339,7 @@ public class TestChecksum {
}
/**
* A class that introduces hbase-checksum failures while
* A class that introduces hbase-checksum failures while
* reading data from hfiles. This should trigger the hdfs level
* checksum validations.
*/
@ -354,4 +356,3 @@ public class TestChecksum {
}
}
}

View File

@ -314,7 +314,7 @@ public class TestHFileBlock {
.withIncludesTags(includesTag)
.withCompression(algo).build();
HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(is, totalSize, meta);
HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
HFileBlock b = hbr.readBlockData(0, -1, pread);
is.close();
assertEquals(0, HFile.getChecksumFailuresCount());
@ -328,17 +328,15 @@ public class TestHFileBlock {
is = fs.open(path);
hbr = new HFileBlock.FSReaderImpl(is, totalSize, meta);
b = hbr.readBlockData(0, 2173 + HConstants.HFILEBLOCK_HEADER_SIZE +
b.totalChecksumBytes(), -1, pread);
b.totalChecksumBytes(), pread);
assertEquals(expected, b);
int wrongCompressedSize = 2172;
try {
b = hbr.readBlockData(0, wrongCompressedSize
+ HConstants.HFILEBLOCK_HEADER_SIZE, -1, pread);
+ HConstants.HFILEBLOCK_HEADER_SIZE, pread);
fail("Exception expected");
} catch (IOException ex) {
String expectedPrefix = "On-disk size without header provided is "
+ wrongCompressedSize + ", but block header contains "
+ b.getOnDiskSizeWithoutHeader() + ".";
String expectedPrefix = "Passed in onDiskSizeWithHeader=";
assertTrue("Invalid exception message: '" + ex.getMessage()
+ "'.\nMessage is expected to start with: '" + expectedPrefix
+ "'", ex.getMessage().startsWith(expectedPrefix));
@ -418,7 +416,7 @@ public class TestHFileBlock {
HFileBlock blockFromHFile, blockUnpacked;
int pos = 0;
for (int blockId = 0; blockId < numBlocks; ++blockId) {
blockFromHFile = hbr.readBlockData(pos, -1, -1, pread);
blockFromHFile = hbr.readBlockData(pos, -1, pread);
assertEquals(0, HFile.getChecksumFailuresCount());
blockFromHFile.sanityCheck();
pos += blockFromHFile.getOnDiskSizeWithHeader();
@ -552,7 +550,7 @@ public class TestHFileBlock {
if (detailedLogging) {
LOG.info("Reading block #" + i + " at offset " + curOffset);
}
HFileBlock b = hbr.readBlockData(curOffset, -1, -1, pread);
HFileBlock b = hbr.readBlockData(curOffset, -1, pread);
if (detailedLogging) {
LOG.info("Block #" + i + ": " + b);
}
@ -566,8 +564,7 @@ public class TestHFileBlock {
// 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);
HFileBlock b2 = hbr.readBlockData(curOffset, b.getOnDiskSizeWithHeader(), pread);
b2.sanityCheck();
assertEquals(b.getBlockType(), b2.getBlockType());
@ -593,7 +590,7 @@ public class TestHFileBlock {
b = b.unpack(meta, hbr);
// b's buffer has header + data + checksum while
// expectedContents have header + data only
ByteBuffer bufRead = b.getBufferWithHeader();
ByteBuffer bufRead = b.getBufferReadOnly();
ByteBuffer bufExpected = expectedContents.get(i);
boolean bytesAreCorrect = Bytes.compareTo(bufRead.array(),
bufRead.arrayOffset(),
@ -676,7 +673,7 @@ public class TestHFileBlock {
HFileBlock b;
try {
long onDiskSizeArg = withOnDiskSize ? expectedSize : -1;
b = hbr.readBlockData(offset, onDiskSizeArg, -1, pread);
b = hbr.readBlockData(offset, onDiskSizeArg, pread);
} catch (IOException ex) {
LOG.error("Error in client " + clientId + " trying to read block at "
+ offset + ", pread=" + pread + ", withOnDiskSize=" +
@ -711,8 +708,7 @@ public class TestHFileBlock {
protected void testConcurrentReadingInternals() throws IOException,
InterruptedException, ExecutionException {
for (Compression.Algorithm compressAlgo : COMPRESSION_ALGORITHMS) {
Path path =
new Path(TEST_UTIL.getDataTestDir(), "concurrent_reading");
Path path = new Path(TEST_UTIL.getDataTestDir(), "concurrent_reading");
Random rand = defaultRandom();
List<Long> offsets = new ArrayList<Long>();
List<BlockType> types = new ArrayList<BlockType>();
@ -835,8 +831,7 @@ public class TestHFileBlock {
.withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
.withChecksumType(ChecksumType.NULL).build();
HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
HFileBlock.FILL_HEADER, -1,
0, meta);
HFileBlock.FILL_HEADER, -1, 0, -1, meta);
long byteBufferExpectedSize =
ClassSize.align(ClassSize.estimateBase(buf.getClass(), true)
+ HConstants.HFILEBLOCK_HEADER_SIZE + size);

View File

@ -1,747 +0,0 @@
/*
*
* 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.apache.hadoop.hbase.io.compress.Compression.Algorithm.GZ;
import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.NONE;
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.Collection;
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.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ChecksumType;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.io.compress.Compressor;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
import com.google.common.base.Preconditions;
/**
* This class has unit tests to prove that older versions of
* HFiles (without checksums) are compatible with current readers.
*/
@Category(SmallTests.class)
@RunWith(Parameterized.class)
public class TestHFileBlockCompatibility {
private static final Log LOG = LogFactory.getLog(TestHFileBlockCompatibility.class);
private static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = {
NONE, GZ };
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private HFileSystem fs;
private final boolean includesMemstoreTS;
private final boolean includesTag;
public TestHFileBlockCompatibility(boolean includesMemstoreTS, boolean includesTag) {
this.includesMemstoreTS = includesMemstoreTS;
this.includesTag = includesTag;
}
@Parameters
public static Collection<Object[]> parameters() {
return HBaseTestingUtility.MEMSTORETS_TAGS_PARAMETRIZED;
}
@Before
public void setUp() throws IOException {
fs = (HFileSystem)HFileSystem.get(TEST_UTIL.getConfiguration());
}
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.
TestHFileBlock.writeTestBlockContents(dos);
dos.flush();
algo.returnCompressor(compressor);
return baos.toByteArray();
}
private Writer createTestV2Block(Compression.Algorithm algo)
throws IOException {
final BlockType blockType = BlockType.DATA;
Writer hbw = new Writer(algo, null,
includesMemstoreTS, includesTag);
DataOutputStream dos = hbw.startWriting(blockType);
TestHFileBlock.writeTestBlockContents(dos);
// make sure the block is ready by calling hbw.getHeaderAndData()
hbw.getHeaderAndData();
assertEquals(1000 * 4, hbw.getUncompressedSizeWithoutHeader());
hbw.releaseCompressor();
return hbw;
}
private String createTestBlockStr(Compression.Algorithm algo,
int correctLength) throws IOException {
Writer hbw = createTestV2Block(algo);
byte[] testV2Block = hbw.getHeaderAndData();
int osOffset = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM + 9;
if (testV2Block.length == correctLength) {
// 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, createTestV2Block(NONE).getBlockForCaching().
getUncompressedSizeWithoutHeader());
}
@Test
public void testGzipCompression() throws IOException {
final String correctTestBlockStr =
"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\\x5Cs\\xA0\\x0F\\x00\\x00";
final int correctGzipBlockLength = 82;
String returnedStr = createTestBlockStr(GZ, correctGzipBlockLength);
assertEquals(correctTestBlockStr, returnedStr);
}
@Test
public void testReaderV2() throws IOException {
if(includesTag) {
TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
}
for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
for (boolean pread : new boolean[] { false, true }) {
LOG.info("testReaderV2: Compression algorithm: " + algo +
", pread=" + pread);
Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
+ algo);
FSDataOutputStream os = fs.create(path);
Writer hbw = new Writer(algo, null,
includesMemstoreTS, includesTag);
long totalSize = 0;
for (int blockId = 0; blockId < 2; ++blockId) {
DataOutputStream dos = hbw.startWriting(BlockType.DATA);
for (int i = 0; i < 1234; ++i)
dos.writeInt(i);
hbw.writeHeaderAndData(os);
totalSize += hbw.getOnDiskSizeWithHeader();
}
os.close();
FSDataInputStream is = fs.open(path);
HFileContext meta = new HFileContextBuilder()
.withHBaseCheckSum(false)
.withIncludesMvcc(includesMemstoreTS)
.withIncludesTags(includesTag)
.withCompression(algo)
.build();
HFileBlock.FSReader hbr =
new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is), totalSize, fs, path, meta);
HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
is.close();
b.sanityCheck();
assertEquals(4936, b.getUncompressedSizeWithoutHeader());
assertEquals(algo == GZ ? 2173 : 4936,
b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
HFileBlock expected = b;
if (algo == GZ) {
is = fs.open(path);
hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is), totalSize, fs, path,
meta);
b = hbr.readBlockData(0, 2173 + HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM +
b.totalChecksumBytes(), -1, pread);
assertEquals(expected, b);
int wrongCompressedSize = 2172;
try {
b = hbr.readBlockData(0, wrongCompressedSize
+ HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM, -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 encoding/decoding data blocks.
* @throws IOException a bug or a problem with temporary files.
*/
@Test
public void testDataBlockEncoding() throws IOException {
if(includesTag) {
TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
}
final int numBlocks = 5;
for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
for (boolean pread : new boolean[] { false, true }) {
for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
LOG.info("testDataBlockEncoding algo " + algo +
" pread = " + pread +
" encoding " + encoding);
Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
+ algo + "_" + encoding.toString());
FSDataOutputStream os = fs.create(path);
HFileDataBlockEncoder dataBlockEncoder = (encoding != DataBlockEncoding.NONE) ?
new HFileDataBlockEncoderImpl(encoding) : NoOpDataBlockEncoder.INSTANCE;
TestHFileBlockCompatibility.Writer hbw =
new TestHFileBlockCompatibility.Writer(algo,
dataBlockEncoder, includesMemstoreTS, includesTag);
long totalSize = 0;
final List<Integer> encodedSizes = new ArrayList<Integer>();
final List<ByteBuffer> encodedBlocks = new ArrayList<ByteBuffer>();
for (int blockId = 0; blockId < numBlocks; ++blockId) {
hbw.startWriting(BlockType.DATA);
TestHFileBlock.writeTestKeyValues(hbw, blockId, pread, includesTag);
hbw.writeHeaderAndData(os);
int headerLen = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
byte[] encodedResultWithHeader = hbw.getUncompressedDataWithHeader();
final int encodedSize = encodedResultWithHeader.length - headerLen;
if (encoding != DataBlockEncoding.NONE) {
// We need to account for the two-byte encoding algorithm ID that
// comes after the 24-byte block header but before encoded KVs.
headerLen += DataBlockEncoding.ID_SIZE;
}
byte[] encodedDataSection =
new byte[encodedResultWithHeader.length - headerLen];
System.arraycopy(encodedResultWithHeader, headerLen,
encodedDataSection, 0, encodedDataSection.length);
final ByteBuffer encodedBuf =
ByteBuffer.wrap(encodedDataSection);
encodedSizes.add(encodedSize);
encodedBlocks.add(encodedBuf);
totalSize += hbw.getOnDiskSizeWithHeader();
}
os.close();
FSDataInputStream is = fs.open(path);
HFileContext meta = new HFileContextBuilder()
.withHBaseCheckSum(false)
.withIncludesMvcc(includesMemstoreTS)
.withIncludesTags(includesTag)
.withCompression(algo)
.build();
HFileBlock.FSReaderImpl hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(is),
totalSize, fs, path, meta);
hbr.setDataBlockEncoder(dataBlockEncoder);
hbr.setIncludesMemstoreTS(includesMemstoreTS);
HFileBlock b;
int pos = 0;
for (int blockId = 0; blockId < numBlocks; ++blockId) {
b = hbr.readBlockData(pos, -1, -1, pread);
b.sanityCheck();
if (meta.isCompressedOrEncrypted()) {
assertFalse(b.isUnpacked());
b = b.unpack(meta, hbr);
}
pos += b.getOnDiskSizeWithHeader();
assertEquals((int) encodedSizes.get(blockId),
b.getUncompressedSizeWithoutHeader());
ByteBuffer actualBuffer = b.getBufferWithoutHeader();
if (encoding != DataBlockEncoding.NONE) {
// We expect a two-byte big-endian encoding id.
assertEquals(0, actualBuffer.get(0));
assertEquals(encoding.getId(), actualBuffer.get(1));
actualBuffer.position(2);
actualBuffer = actualBuffer.slice();
}
ByteBuffer expectedBuffer = encodedBlocks.get(blockId);
expectedBuffer.rewind();
// test if content matches, produce nice message
TestHFileBlock.assertBuffersEqual(expectedBuffer, actualBuffer,
algo, encoding, pread);
}
is.close();
}
}
}
}
/**
* This is the version of the HFileBlock.Writer that is used to
* create V2 blocks with minor version 0. These blocks do not
* have hbase-level checksums. The code is here to test
* backward compatibility. The reason we do not inherit from
* HFileBlock.Writer is because we never ever want to change the code
* in this class but the code in HFileBlock.Writer will continually
* evolve.
*/
public static final class Writer extends HFileBlock.Writer {
// These constants are as they were in minorVersion 0.
private static final int HEADER_SIZE = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
private static final boolean DONT_FILL_HEADER = HFileBlock.DONT_FILL_HEADER;
private static final byte[] DUMMY_HEADER = HFileBlock.DUMMY_HEADER_NO_CHECKSUM;
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;
/** Data block encoder used for data blocks */
private final HFileDataBlockEncoder dataBlockEncoder;
private HFileBlockEncodingContext dataBlockEncodingCtx;
/** block encoding context for non-data blocks */
private HFileBlockDefaultEncodingContext defaultBlockEncodingCtx;
/**
* The stream we use to accumulate data in uncompressed format for each
* block. 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 baosInMemory;
/** Compressor, which is also reused between consecutive blocks. */
private Compressor compressor;
/**
* Current block type. Set in {@link #startWriting(BlockType)}. Could be
* changed in {@link #encodeDataBlockForDisk()} from {@link BlockType#DATA}
* to {@link BlockType#ENCODED_DATA}.
*/
private BlockType blockType;
/**
* A stream that we write uncompressed bytes to, which compresses them and
* writes them to {@link #baosInMemory}.
*/
private DataOutputStream userDataStream;
/**
* Bytes to be written to the file system, including the header. Compressed
* if compression is turned on.
*/
private byte[] onDiskBytesWithHeader;
/**
* Valid in the READY state. Contains the header and the uncompressed (but
* potentially encoded, if this is a data block) bytes, so the length is
* {@link #uncompressedSizeWithoutHeader} + {@link org.apache.hadoop.hbase.HConstants#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;
/** The offset of the previous block of the same type */
private long prevOffset;
private int unencodedDataSizeWritten;
public Writer(Compression.Algorithm compressionAlgorithm,
HFileDataBlockEncoder dataBlockEncoder, boolean includesMemstoreTS, boolean includesTag) {
this(dataBlockEncoder, new HFileContextBuilder().withHBaseCheckSum(false)
.withIncludesMvcc(includesMemstoreTS).withIncludesTags(includesTag)
.withCompression(compressionAlgorithm).build());
}
public Writer(HFileDataBlockEncoder dataBlockEncoder, HFileContext meta) {
super(dataBlockEncoder, meta);
compressAlgo = meta.getCompression() == null ? NONE : meta.getCompression();
this.dataBlockEncoder = dataBlockEncoder != null ? dataBlockEncoder
: NoOpDataBlockEncoder.INSTANCE;
defaultBlockEncodingCtx = new HFileBlockDefaultEncodingContext(null, DUMMY_HEADER, meta);
dataBlockEncodingCtx = this.dataBlockEncoder.newDataBlockEncodingContext(DUMMY_HEADER, meta);
baosInMemory = new ByteArrayOutputStream();
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)
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.getId()] = startOffset;
}
startOffset = -1;
blockType = newBlockType;
baosInMemory.reset();
baosInMemory.write(DUMMY_HEADER);
state = State.WRITING;
// We will compress it later in finishBlock()
userDataStream = new DataOutputStream(baosInMemory);
if (newBlockType == BlockType.DATA) {
this.dataBlockEncoder.startBlockEncoding(dataBlockEncodingCtx, userDataStream);
}
this.unencodedDataSizeWritten = 0;
return userDataStream;
}
@Override
public void write(Cell c) throws IOException {
KeyValue kv = KeyValueUtil.ensureKeyValue(c);
expectState(State.WRITING);
this.dataBlockEncoder.encode(kv, dataBlockEncodingCtx, this.userDataStream);
this.unencodedDataSizeWritten += kv.getLength();
if (dataBlockEncodingCtx.getHFileContext().isIncludesMvcc()) {
this.unencodedDataSizeWritten += WritableUtils.getVIntSize(kv.getMvccVersion());
}
}
/**
* 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.
*/
void ensureBlockReady() throws IOException {
Preconditions.checkState(state != State.INIT,
"Unexpected state: " + state);
if (state == State.BLOCK_READY)
return;
// This will set state to BLOCK_READY.
finishBlock();
}
/**
* 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. Sets block
* write state to "block ready".
*/
void finishBlock() throws IOException {
if (blockType == BlockType.DATA) {
this.dataBlockEncoder.endBlockEncoding(dataBlockEncodingCtx, userDataStream,
baosInMemory.toByteArray(), blockType);
blockType = dataBlockEncodingCtx.getBlockType();
}
userDataStream.flush();
// This does an array copy, so it is safe to cache this byte array.
uncompressedBytesWithHeader = baosInMemory.toByteArray();
prevOffset = prevOffsetByType[blockType.getId()];
// We need to set state before we can package the block up for
// cache-on-write. In a way, the block is ready, but not yet encoded or
// compressed.
state = State.BLOCK_READY;
if (blockType == BlockType.DATA || blockType == BlockType.ENCODED_DATA) {
onDiskBytesWithHeader = dataBlockEncodingCtx
.compressAndEncrypt(uncompressedBytesWithHeader);
} else {
onDiskBytesWithHeader = defaultBlockEncodingCtx
.compressAndEncrypt(uncompressedBytesWithHeader);
}
// put the header for on disk bytes
putHeader(onDiskBytesWithHeader, 0,
onDiskBytesWithHeader.length,
uncompressedBytesWithHeader.length);
//set the header for the uncompressed bytes (for cache-on-write)
putHeader(uncompressedBytesWithHeader, 0,
onDiskBytesWithHeader.length,
uncompressedBytesWithHeader.length);
}
/**
* Put the header into the given byte array at the given offset.
* @param onDiskSize size of the block on disk
* @param uncompressedSize size of the block after decompression (but
* before optional data block decoding)
*/
private void putHeader(byte[] dest, int offset, int onDiskSize,
int uncompressedSize) {
offset = blockType.put(dest, offset);
offset = Bytes.putInt(dest, offset, onDiskSize - HEADER_SIZE);
offset = Bytes.putInt(dest, offset, uncompressedSize - HEADER_SIZE);
Bytes.putLong(dest, offset, prevOffset);
}
/**
* Similar to {@link #writeHeaderAndData(FSDataOutputStream)}, 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 uncompressedBytesWithHeader.length - HEADER_SIZE;
}
/**
* The uncompressed size of the block data, including header size.
*/
public int getUncompressedSizeWithHeader() {
expectState(State.BLOCK_READY);
return uncompressedBytesWithHeader.length;
}
/** @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 this.unencodedDataSizeWritten;
}
/**
* 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);
return uncompressedBytesWithHeader;
}
private void expectState(State expectedState) {
if (state != expectedState) {
throw new IllegalStateException("Expected state: " + expectedState +
", actual state: " + state);
}
}
/**
* Similar to {@link #getUncompressedBufferWithHeader()} 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()));
writeHeaderAndData(out);
}
/**
* Creates a new HFileBlock.
*/
public HFileBlock getBlockForCaching() {
HFileContext meta = new HFileContextBuilder()
.withHBaseCheckSum(false)
.withChecksumType(ChecksumType.NULL)
.withBytesPerCheckSum(0)
.build();
return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
getUncompressedSizeWithoutHeader(), prevOffset,
getUncompressedBufferWithHeader(), DONT_FILL_HEADER, startOffset,
getOnDiskSizeWithoutHeader(), meta);
}
}
}

View File

@ -176,8 +176,7 @@ public class TestHFileBlockIndex {
}
missCount += 1;
prevBlock = realReader.readBlockData(offset, onDiskSize,
-1, pread);
prevBlock = realReader.readBlockData(offset, onDiskSize, pread);
prevOffset = offset;
prevOnDiskSize = onDiskSize;
prevPread = pread;

View File

@ -90,8 +90,7 @@ public class TestHFileDataBlockEncoder {
if (blockEncoder.getDataBlockEncoding() ==
DataBlockEncoding.NONE) {
assertEquals(block.getBufferWithHeader(),
returnedBlock.getBufferWithHeader());
assertEquals(block.getBufferReadOnly(), returnedBlock.getBufferReadOnly());
} else {
if (BlockType.ENCODED_DATA != returnedBlock.getBlockType()) {
System.out.println(blockEncoder);
@ -125,7 +124,7 @@ public class TestHFileDataBlockEncoder {
.build();
HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
HFileBlock.FILL_HEADER, 0,
0, hfileContext);
0, -1, hfileContext);
HFileBlock cacheBlock = createBlockOnDisk(kvs, block, useTags);
assertEquals(headerSize, cacheBlock.getDummyHeaderForVersion().length);
}
@ -172,8 +171,8 @@ public class TestHFileDataBlockEncoder {
.withChecksumType(ChecksumType.NULL)
.build();
HFileBlock b = new HFileBlock(BlockType.DATA, size, size, -1, buf,
HFileBlock.FILL_HEADER, 0,
0, meta);
HFileBlock.FILL_HEADER, 0,
0, -1, meta);
return b;
}
@ -197,7 +196,8 @@ public class TestHFileDataBlockEncoder {
byte[] encodedBytes = baos.toByteArray();
size = encodedBytes.length - block.getDummyHeaderForVersion().length;
return new HFileBlock(context.getBlockType(), size, size, -1, ByteBuffer.wrap(encodedBytes),
HFileBlock.FILL_HEADER, 0, block.getOnDiskDataSizeWithHeader(), block.getHFileContext());
HFileBlock.FILL_HEADER, 0, block.getOnDiskDataSizeWithHeader(), -1,
block.getHFileContext());
}
/**
@ -210,7 +210,7 @@ public class TestHFileDataBlockEncoder {
for (DataBlockEncoding diskAlgo : DataBlockEncoding.values()) {
for (boolean includesMemstoreTS : new boolean[] { false, true }) {
HFileDataBlockEncoder dbe = (diskAlgo == DataBlockEncoding.NONE) ?
HFileDataBlockEncoder dbe = (diskAlgo == DataBlockEncoding.NONE) ?
NoOpDataBlockEncoder.INSTANCE : new HFileDataBlockEncoderImpl(diskAlgo);
configurations.add(new Object[] { dbe, new Boolean(includesMemstoreTS) });
}

View File

@ -98,7 +98,7 @@ public class TestHFileEncryption {
private long readAndVerifyBlock(long pos, HFileContext ctx, HFileBlock.FSReaderImpl hbr, int size)
throws IOException {
HFileBlock b = hbr.readBlockData(pos, -1, -1, false);
HFileBlock b = hbr.readBlockData(pos, -1, false);
assertEquals(0, HFile.getChecksumFailuresCount());
b.sanityCheck();
assertFalse(b.isUnpacked());

View File

@ -124,7 +124,7 @@ public class TestHFileWriterV2 {
writer.appendMetaBlock("CAPITAL_OF_FRANCE", new Text("Paris"));
writer.close();
FSDataInputStream fsdis = fs.open(hfilePath);
@ -144,7 +144,7 @@ public class TestHFileWriterV2 {
.withIncludesTags(false)
.withCompression(compressAlgo)
.build();
HFileBlock.FSReader blockReader = new HFileBlock.FSReaderImpl(fsdis, fileSize, meta);
// Comparator class name is stored in the trailer in version 2.
KVComparator comparator = trailer.createComparator();
@ -163,12 +163,12 @@ public class TestHFileWriterV2 {
dataBlockIndexReader.readMultiLevelIndexRoot(
blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX),
trailer.getDataIndexCount());
if (findMidKey) {
byte[] midkey = dataBlockIndexReader.midkey();
assertNotNull("Midkey should not be null", midkey);
}
// Meta index.
metaBlockIndexReader.readRootIndex(
blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX)
@ -190,7 +190,7 @@ public class TestHFileWriterV2 {
fsdis.seek(0);
long curBlockPos = 0;
while (curBlockPos <= trailer.getLastDataBlockOffset()) {
HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false);
HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false);
assertEquals(BlockType.DATA, block.getBlockType());
if (meta.isCompressedOrEncrypted()) {
assertFalse(block.isUnpacked());
@ -237,17 +237,18 @@ public class TestHFileWriterV2 {
while (fsdis.getPos() < trailer.getLoadOnOpenDataOffset()) {
LOG.info("Current offset: " + fsdis.getPos() + ", scanning until " +
trailer.getLoadOnOpenDataOffset());
HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false)
.unpack(meta, blockReader);
assertEquals(BlockType.META, block.getBlockType());
Text t = new Text();
ByteBuffer buf = block.getBufferWithoutHeader();
if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), t) == null) {
throw new IOException("Failed to deserialize block " + this + " into a " + t.getClass().getSimpleName());
throw new IOException("Failed to deserialize block " + this + " into a " +t.getClass().getSimpleName());
}
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;

View File

@ -159,7 +159,7 @@ public class TestHFileWriterV3 {
writer.appendMetaBlock("CAPITAL_OF_FRANCE", new Text("Paris"));
writer.close();
FSDataInputStream fsdis = fs.open(hfilePath);
@ -192,12 +192,12 @@ public class TestHFileWriterV3 {
// the root level.
dataBlockIndexReader.readMultiLevelIndexRoot(
blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX), trailer.getDataIndexCount());
if (findMidKey) {
byte[] midkey = dataBlockIndexReader.midkey();
assertNotNull("Midkey should not be null", midkey);
}
// Meta index.
metaBlockIndexReader.readRootIndex(
blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX)
@ -219,7 +219,7 @@ public class TestHFileWriterV3 {
fsdis.seek(0);
long curBlockPos = 0;
while (curBlockPos <= trailer.getLastDataBlockOffset()) {
HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false)
.unpack(context, blockReader);
assertEquals(BlockType.DATA, block.getBlockType());
ByteBuffer buf = block.getBufferWithoutHeader();
@ -241,7 +241,7 @@ public class TestHFileWriterV3 {
tagValue = new byte[tagLen];
buf.get(tagValue);
}
if (includeMemstoreTS) {
ByteArrayInputStream byte_input = new ByteArrayInputStream(buf.array(), buf.arrayOffset()
+ buf.position(), buf.remaining());
@ -278,13 +278,14 @@ public class TestHFileWriterV3 {
while (fsdis.getPos() < trailer.getLoadOnOpenDataOffset()) {
LOG.info("Current offset: " + fsdis.getPos() + ", scanning until " +
trailer.getLoadOnOpenDataOffset());
HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
HFileBlock block = blockReader.readBlockData(curBlockPos, -1, false)
.unpack(context, blockReader);
assertEquals(BlockType.META, block.getBlockType());
Text t = new Text();
ByteBuffer buf = block.getBufferWithoutHeader();
if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), t) == null) {
throw new IOException("Failed to deserialize block " + this + " into a " + t.getClass().getSimpleName());
throw new IOException("Failed to deserialize block " + this +
" into a " + t.getClass().getSimpleName());
}
Text expectedText =
(metaCounter == 0 ? new Text("Paris") : metaCounter == 1 ? new Text(

View File

@ -91,14 +91,8 @@ public class TestPrefetch {
// Check that all of the data blocks were preloaded
BlockCache blockCache = cacheConf.getBlockCache();
long offset = 0;
HFileBlock prevBlock = null;
while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
long onDiskSize = -1;
if (prevBlock != null) {
onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
}
HFileBlock block = reader.readBlock(offset, onDiskSize, false, true, false, true, null,
null);
HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null, null);
BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(), offset);
boolean isCached = blockCache.getBlock(blockCacheKey, true, false, true) != null;
if (block.getBlockType() == BlockType.DATA ||
@ -106,7 +100,6 @@ public class TestPrefetch {
block.getBlockType() == BlockType.INTERMEDIATE_INDEX) {
assertTrue(isCached);
}
prevBlock = block;
offset += block.getOnDiskSizeWithHeader();
}
}

View File

@ -1,348 +0,0 @@
/**
* Copyright 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.migration;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
import java.io.File;
import java.io.IOException;
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.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
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.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.util.ToolRunner;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test upgrade from no namespace in 0.94 to namespace directory structure.
* Mainly tests that tables are migrated and consistent. Also verifies
* that snapshots have been migrated correctly.
*
* <p>Uses a tarball which is an image of an 0.94 hbase.rootdir.
*
* <p>Contains tables with currentKeys as the stored keys:
* foo, ns1.foo, ns2.foo
*
* <p>Contains snapshots with snapshot{num}Keys as the contents:
* snapshot1Keys, snapshot2Keys
*
* Image also contains _acl_ table with one region and two storefiles.
* This is needed to test the acl table migration.
*
*/
@Category(MediumTests.class)
public class TestNamespaceUpgrade {
static final Log LOG = LogFactory.getLog(TestNamespaceUpgrade.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final static String snapshot1Keys[] =
{"1","10","2","3","4","5","6","7","8","9"};
private final static String snapshot2Keys[] =
{"1","2","3","4","5","6","7","8","9"};
private final static String currentKeys[] =
{"1","2","3","4","5","6","7","8","9","A"};
private final static TableName tables[] =
{TableName.valueOf("data"), TableName.valueOf("foo"),
TableName.valueOf("ns1.foo"), TableName.valueOf("ns.two.foo")};
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Start up our mini cluster on top of an 0.94 root.dir that has data from
// a 0.94 hbase run and see if we can migrate to 0.96
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.startMiniDFSCluster(1);
Path testdir = TEST_UTIL.getDataTestDir("TestNamespaceUpgrade");
// Untar our test dir.
File untar = untar(new File(testdir.toString()));
// Now copy the untar up into hdfs so when we start hbase, we'll run from it.
Configuration conf = TEST_UTIL.getConfiguration();
FsShell shell = new FsShell(conf);
FileSystem fs = FileSystem.get(conf);
// find where hbase will root itself, so we can copy filesystem there
Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
if (!fs.isDirectory(hbaseRootDir.getParent())) {
// mkdir at first
fs.mkdirs(hbaseRootDir.getParent());
}
if(org.apache.hadoop.util.VersionInfo.getVersion().startsWith("2.")) {
LOG.info("Hadoop version is 2.x, pre-migrating snapshot dir");
FileSystem localFS = FileSystem.getLocal(conf);
if(!localFS.rename(new Path(untar.toString(), HConstants.OLD_SNAPSHOT_DIR_NAME),
new Path(untar.toString(), HConstants.SNAPSHOT_DIR_NAME))) {
throw new IllegalStateException("Failed to move snapshot dir to 2.x expectation");
}
}
doFsCommand(shell,
new String [] {"-put", untar.toURI().toString(), hbaseRootDir.toString()});
doFsCommand(shell, new String [] {"-lsr", "/"});
// See whats in minihdfs.
Configuration toolConf = TEST_UTIL.getConfiguration();
conf.set(HConstants.HBASE_DIR, TEST_UTIL.getDefaultRootDirPath().toString());
ToolRunner.run(toolConf, new NamespaceUpgrade(), new String[]{"--upgrade"});
assertTrue(FSUtils.getVersion(fs, hbaseRootDir).equals(HConstants.FILE_SYSTEM_VERSION));
doFsCommand(shell, new String [] {"-lsr", "/"});
TEST_UTIL.startMiniHBaseCluster(1, 1);
for(TableName table: tables) {
int count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), table).getScanner(new Scan())) {
assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
}
Assert.assertEquals(currentKeys.length, count);
}
assertEquals(2, TEST_UTIL.getHBaseAdmin().listNamespaceDescriptors().length);
//verify ACL table is migrated
HTable secureTable = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
ResultScanner scanner = secureTable.getScanner(new Scan());
int count = 0;
for(Result r : scanner) {
count++;
}
assertEquals(3, count);
assertFalse(TEST_UTIL.getHBaseAdmin().tableExists(TableName.valueOf("_acl_")));
//verify ACL table was compacted
List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(secureTable.getName());
for(HRegion region : regions) {
assertEquals(1, region.getStores().size());
}
}
static File untar(final File testdir) throws IOException {
// Find the src data under src/test/data
final String datafile = "TestNamespaceUpgrade";
File srcTarFile = new File(
System.getProperty("project.build.testSourceDirectory", "src/test") +
File.separator + "data" + File.separator + datafile + ".tgz");
File homedir = new File(testdir.toString());
File tgtUntarDir = new File(homedir, "hbase");
if (tgtUntarDir.exists()) {
if (!FileUtil.fullyDelete(tgtUntarDir)) {
throw new IOException("Failed delete of " + tgtUntarDir.toString());
}
}
if (!srcTarFile.exists()) {
throw new IOException(srcTarFile+" does not exist");
}
LOG.info("Untarring " + srcTarFile + " into " + homedir.toString());
FileUtil.unTar(srcTarFile, homedir);
Assert.assertTrue(tgtUntarDir.exists());
return tgtUntarDir;
}
private static void doFsCommand(final FsShell shell, final String [] args)
throws Exception {
// Run the 'put' command.
int errcode = shell.run(args);
if (errcode != 0) throw new IOException("Failed put; errcode=" + errcode);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Test (timeout=300000)
public void testSnapshots() throws IOException, InterruptedException {
String snapshots[][] = {snapshot1Keys, snapshot2Keys};
for(int i = 1; i <= snapshots.length; i++) {
for(TableName table: tables) {
TEST_UTIL.getHBaseAdmin().cloneSnapshot(table+"_snapshot"+i, TableName.valueOf(table+"_clone"+i));
FSUtils.logFileSystemState(FileSystem.get(TEST_UTIL.getConfiguration()),
FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
LOG);
int count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), table+"_clone"+i).getScanner(new
Scan())) {
assertEquals(snapshots[i-1][count++], Bytes.toString(res.getRow()));
}
Assert.assertEquals(table+"_snapshot"+i, snapshots[i-1].length, count);
}
}
}
@Test (timeout=300000)
public void testRenameUsingSnapshots() throws Exception {
String newNS = "newNS";
TEST_UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(newNS).build());
for(TableName table: tables) {
int count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), table).getScanner(new
Scan())) {
assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
}
TEST_UTIL.getHBaseAdmin().snapshot(table + "_snapshot3", table);
final TableName newTableName =
TableName.valueOf(newNS + TableName.NAMESPACE_DELIM + table + "_clone3");
TEST_UTIL.getHBaseAdmin().cloneSnapshot(table + "_snapshot3", newTableName);
Thread.sleep(1000);
count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), newTableName).getScanner(new
Scan())) {
assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
}
FSUtils.logFileSystemState(TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath()
, LOG);
Assert.assertEquals(newTableName + "", currentKeys.length, count);
TEST_UTIL.getHBaseAdmin().flush(newTableName);
TEST_UTIL.getHBaseAdmin().majorCompact(newTableName);
TEST_UTIL.waitFor(30000, new Waiter.Predicate<IOException>() {
@Override
public boolean evaluate() throws IOException {
return TEST_UTIL.getHBaseAdmin().getCompactionState(newTableName) ==
AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
}
});
}
String nextNS = "nextNS";
TEST_UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(nextNS).build());
for(TableName table: tables) {
TableName srcTable = TableName.valueOf(newNS + TableName.NAMESPACE_DELIM + table + "_clone3");
TEST_UTIL.getHBaseAdmin().snapshot(table + "_snapshot4", srcTable);
TableName newTableName =
TableName.valueOf(nextNS + TableName.NAMESPACE_DELIM + table + "_clone4");
TEST_UTIL.getHBaseAdmin().cloneSnapshot(table+"_snapshot4", newTableName);
FSUtils.logFileSystemState(TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath(),
LOG);
int count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), newTableName).getScanner(new
Scan())) {
assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
}
Assert.assertEquals(newTableName + "", currentKeys.length, count);
}
}
@Test (timeout=300000)
public void testOldDirsAreGonePostMigration() throws IOException {
FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
List <String> dirs = new ArrayList<String>(NamespaceUpgrade.NON_USER_TABLE_DIRS);
// Remove those that are not renamed
dirs.remove(HConstants.HBCK_SIDELINEDIR_NAME);
dirs.remove(HConstants.SNAPSHOT_DIR_NAME);
dirs.remove(HConstants.HBASE_TEMP_DIRECTORY);
for (String dir: dirs) {
assertFalse(fs.exists(new Path(hbaseRootDir, dir)));
}
}
@Test (timeout=300000)
public void testNewDirsArePresentPostMigration() throws IOException {
FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
// Below list does not include 'corrupt' because there is no 'corrupt' in the tgz
String [] newdirs = new String [] {HConstants.BASE_NAMESPACE_DIR,
HConstants.HREGION_LOGDIR_NAME};
Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
for (String dir: newdirs) {
assertTrue(dir, fs.exists(new Path(hbaseRootDir, dir)));
}
}
@Test (timeout = 300000)
public void testACLTableMigration() throws IOException {
Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("testACLTable");
FileSystem fs = TEST_UTIL.getTestFileSystem();
Configuration conf = TEST_UTIL.getConfiguration();
byte[] FAMILY = Bytes.toBytes("l");
byte[] QUALIFIER = Bytes.toBytes("testUser");
byte[] VALUE = Bytes.toBytes("RWCA");
// Create a Region
HTableDescriptor aclTable = new HTableDescriptor(TableName.valueOf("testACLTable"));
aclTable.addFamily(new HColumnDescriptor(FAMILY));
FSTableDescriptors fstd = new FSTableDescriptors(conf, fs, rootDir);
fstd.createTableDescriptor(aclTable);
HRegionInfo hriAcl = new HRegionInfo(aclTable.getTableName(), null, null);
HRegion region = HRegion.createHRegion(hriAcl, rootDir, conf, aclTable);
try {
// Create rows
Put p = new Put(Bytes.toBytes("-ROOT-"));
p.addImmutable(FAMILY, QUALIFIER, VALUE);
region.put(p);
p = new Put(Bytes.toBytes(".META."));
p.addImmutable(FAMILY, QUALIFIER, VALUE);
region.put(p);
p = new Put(Bytes.toBytes("_acl_"));
p.addImmutable(FAMILY, QUALIFIER, VALUE);
region.put(p);
NamespaceUpgrade upgrade = new NamespaceUpgrade();
upgrade.updateAcls(region);
// verify rows -ROOT- is removed
Get g = new Get(Bytes.toBytes("-ROOT-"));
Result r = region.get(g);
assertTrue(r == null || r.size() == 0);
// verify rows _acl_ is renamed to hbase:acl
g = new Get(AccessControlLists.ACL_TABLE_NAME.toBytes());
r = region.get(g);
assertTrue(r != null && r.size() == 1);
assertTrue(Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)) == 0);
// verify rows .META. is renamed to hbase:meta
g = new Get(TableName.META_TABLE_NAME.toBytes());
r = region.get(g);
assertTrue(r != null && r.size() == 1);
assertTrue(Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)) == 0);
} finally {
region.close();
// Delete the region
HRegionFileSystem.deleteRegionFromFileSystem(conf, fs,
FSUtils.getTableDir(rootDir, hriAcl.getTable()), hriAcl);
}
}
}

View File

@ -1,271 +0,0 @@
/**
* 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.migration;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.io.FileLink;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ReplicationPeer;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileV1Detector;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.ToolRunner;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* Upgrade to 0.96 involves detecting HFileV1 in existing cluster, updating namespace and
* updating znodes. This class tests for HFileV1 detection and upgrading znodes.
* Uprading namespace is tested in {@link TestNamespaceUpgrade}.
*/
@Category(MediumTests.class)
public class TestUpgradeTo96 {
static final Log LOG = LogFactory.getLog(TestUpgradeTo96.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
/**
* underlying file system instance
*/
private static FileSystem fs;
/**
* hbase root dir
*/
private static Path hbaseRootDir;
private static ZooKeeperWatcher zkw;
/**
* replication peer znode (/hbase/replication/peers)
*/
private static String replicationPeerZnode;
/**
* znode of a table
*/
private static String tableAZnode;
private static ReplicationPeer peer1;
/**
* znode for replication peer1 (/hbase/replication/peers/1)
*/
private static String peer1Znode;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Start up the mini cluster on top of an 0.94 root.dir that has data from
// a 0.94 hbase run and see if we can migrate to 0.96
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.startMiniDFSCluster(1);
hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
fs = FileSystem.get(TEST_UTIL.getConfiguration());
FSUtils.setRootDir(TEST_UTIL.getConfiguration(), hbaseRootDir);
zkw = TEST_UTIL.getZooKeeperWatcher();
Path testdir = TEST_UTIL.getDataTestDir("TestUpgradeTo96");
// get the untar 0.94 file structure
set94FSLayout(testdir);
setUp94Znodes();
}
/**
* Lays out 0.94 file system layout using {@link TestNamespaceUpgrade} apis.
* @param testdir
* @throws IOException
* @throws Exception
*/
private static void set94FSLayout(Path testdir) throws IOException, Exception {
File untar = TestNamespaceUpgrade.untar(new File(testdir.toString()));
if (!fs.exists(hbaseRootDir.getParent())) {
// mkdir at first
fs.mkdirs(hbaseRootDir.getParent());
}
FsShell shell = new FsShell(TEST_UTIL.getConfiguration());
shell.run(new String[] { "-put", untar.toURI().toString(), hbaseRootDir.toString() });
// See whats in minihdfs.
shell.run(new String[] { "-lsr", "/" });
}
/**
* Sets znodes used in 0.94 version. Only table and replication znodes will be upgraded to PB,
* others would be deleted.
* @throws KeeperException
*/
private static void setUp94Znodes() throws IOException, KeeperException {
// add some old znodes, which would be deleted after upgrade.
String rootRegionServerZnode = ZKUtil.joinZNode(zkw.baseZNode, "root-region-server");
ZKUtil.createWithParents(zkw, rootRegionServerZnode);
ZKUtil.createWithParents(zkw, zkw.backupMasterAddressesZNode);
// add table znode, data of its children would be protobuffized
tableAZnode = ZKUtil.joinZNode(zkw.tableZNode, "a");
ZKUtil.createWithParents(zkw, tableAZnode,
Bytes.toBytes(ZooKeeperProtos.Table.State.ENABLED.toString()));
// add replication znodes, data of its children would be protobuffized
String replicationZnode = ZKUtil.joinZNode(zkw.baseZNode, "replication");
replicationPeerZnode = ZKUtil.joinZNode(replicationZnode, "peers");
peer1Znode = ZKUtil.joinZNode(replicationPeerZnode, "1");
peer1 = ReplicationPeer.newBuilder().setClusterkey("abc:123:/hbase").build();
ZKUtil.createWithParents(zkw, peer1Znode, Bytes.toBytes(peer1.getClusterkey()));
}
/**
* Tests a 0.94 filesystem for any HFileV1.
* @throws Exception
*/
@Test
public void testHFileV1Detector() throws Exception {
assertEquals(0, ToolRunner.run(TEST_UTIL.getConfiguration(), new HFileV1Detector(), null));
}
/**
* Creates a corrupt file, and run HFileV1 detector tool
* @throws Exception
*/
@Test
public void testHFileV1DetectorWithCorruptFiles() throws Exception {
// add a corrupt file.
Path tablePath = new Path(hbaseRootDir, "foo");
FileStatus[] regionsDir = fs.listStatus(tablePath);
if (regionsDir == null) throw new IOException("No Regions found for table " + "foo");
Path columnFamilyDir = null;
Path targetRegion = null;
for (FileStatus s : regionsDir) {
if (fs.exists(new Path(s.getPath(), HRegionFileSystem.REGION_INFO_FILE))) {
targetRegion = s.getPath();
break;
}
}
FileStatus[] cfs = fs.listStatus(targetRegion);
for (FileStatus f : cfs) {
if (f.isDirectory()) {
columnFamilyDir = f.getPath();
break;
}
}
LOG.debug("target columnFamilyDir: " + columnFamilyDir);
// now insert a corrupt file in the columnfamily.
Path corruptFile = new Path(columnFamilyDir, "corrupt_file");
if (!fs.createNewFile(corruptFile)) throw new IOException("Couldn't create corrupt file: "
+ corruptFile);
assertEquals(1, ToolRunner.run(TEST_UTIL.getConfiguration(), new HFileV1Detector(), null));
// remove the corrupt file
FileSystem.get(TEST_UTIL.getConfiguration()).delete(corruptFile, false);
}
@Test
public void testHFileLink() throws Exception {
// pass a link, and verify that correct paths are returned.
Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
Path aFileLink = new Path(rootDir, "table/2086db948c48/cf/table=21212abcdc33-0906db948c48");
Path preNamespaceTablePath = new Path(rootDir, "table/21212abcdc33/cf/0906db948c48");
Path preNamespaceArchivePath =
new Path(rootDir, ".archive/table/21212abcdc33/cf/0906db948c48");
Path preNamespaceTempPath = new Path(rootDir, ".tmp/table/21212abcdc33/cf/0906db948c48");
boolean preNSTablePathExists = false;
boolean preNSArchivePathExists = false;
boolean preNSTempPathExists = false;
assertTrue(HFileLink.isHFileLink(aFileLink));
HFileLink hFileLink =
HFileLink.buildFromHFileLinkPattern(TEST_UTIL.getConfiguration(), aFileLink);
assertTrue(hFileLink.getArchivePath().toString().startsWith(rootDir.toString()));
HFileV1Detector t = new HFileV1Detector();
t.setConf(TEST_UTIL.getConfiguration());
FileLink fileLink = t.getFileLinkWithPreNSPath(aFileLink);
//assert it has 6 paths (2 NS, 2 Pre NS, and 2 .tmp) to look.
assertTrue(fileLink.getLocations().length == 6);
for (Path p : fileLink.getLocations()) {
if (p.equals(preNamespaceArchivePath)) preNSArchivePathExists = true;
if (p.equals(preNamespaceTablePath)) preNSTablePathExists = true;
if (p.equals(preNamespaceTempPath)) preNSTempPathExists = true;
}
assertTrue(preNSArchivePathExists & preNSTablePathExists & preNSTempPathExists);
}
@Test
public void testADirForHFileV1() throws Exception {
Path tablePath = new Path(hbaseRootDir, "foo");
System.out.println("testADirForHFileV1: " + tablePath.makeQualified(fs));
System.out.println("Passed: " + hbaseRootDir + "/foo");
assertEquals(0,
ToolRunner.run(TEST_UTIL.getConfiguration(), new HFileV1Detector(), new String[] { "-p"
+ "foo" }));
}
@Test
public void testZnodeMigration() throws Exception {
String rootRSZnode = ZKUtil.joinZNode(zkw.baseZNode, "root-region-server");
assertTrue(ZKUtil.checkExists(zkw, rootRSZnode) > -1);
ToolRunner.run(TEST_UTIL.getConfiguration(), new UpgradeTo96(), new String[] { "-execute" });
assertEquals(-1, ZKUtil.checkExists(zkw, rootRSZnode));
byte[] data = ZKUtil.getData(zkw, tableAZnode);
assertTrue(ProtobufUtil.isPBMagicPrefix(data));
checkTableState(data, ZooKeeperProtos.Table.State.ENABLED);
// ensure replication znodes are there, and protobuffed.
data = ZKUtil.getData(zkw, peer1Znode);
assertTrue(ProtobufUtil.isPBMagicPrefix(data));
checkReplicationPeerData(data, peer1);
}
private void checkTableState(byte[] data, State expectedState)
throws InvalidProtocolBufferException {
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
int magicLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build();
assertTrue(t.getState() == expectedState);
}
private void checkReplicationPeerData(byte[] data, ReplicationPeer peer)
throws InvalidProtocolBufferException {
int magicLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder();
assertEquals(builder.mergeFrom(data, magicLen, data.length - magicLen).build().getClusterkey(),
peer.getClusterkey());
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL.shutdownMiniDFSCluster();
TEST_UTIL.shutdownMiniZKCluster();
}
}

View File

@ -227,15 +227,10 @@ public class TestCacheOnWriteInSchema {
assertTrue(testDescription, scanner.seekTo());
// Cribbed from io.hfile.TestCacheOnWrite
long offset = 0;
HFileBlock prevBlock = null;
while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
long onDiskSize = -1;
if (prevBlock != null) {
onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
}
// Flags: don't cache the block, use pread, this is not a compaction.
// Also, pass null for expected block type to avoid checking it.
HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
HFileBlock block = reader.readBlock(offset, -1, false, true,
false, true, null, DataBlockEncoding.NONE);
BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
offset);
@ -249,7 +244,6 @@ public class TestCacheOnWriteInSchema {
"block: " + block + "\n" +
"blockCacheKey: " + blockCacheKey);
}
prevBlock = block;
offset += block.getOnDiskSizeWithHeader();
}
} finally {