HBASE-11339 Merge remote-tracking branch 'apache/hbase-11339' (Jingcheng Du)
This commit is contained in:
commit
493f36c899
|
@ -123,6 +123,12 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
public static final String ENCRYPTION = "ENCRYPTION";
|
||||
public static final String ENCRYPTION_KEY = "ENCRYPTION_KEY";
|
||||
|
||||
public static final String IS_MOB = "IS_MOB";
|
||||
public static final byte[] IS_MOB_BYTES = Bytes.toBytes(IS_MOB);
|
||||
public static final String MOB_THRESHOLD = "MOB_THRESHOLD";
|
||||
public static final byte[] MOB_THRESHOLD_BYTES = Bytes.toBytes(MOB_THRESHOLD);
|
||||
public static final long DEFAULT_MOB_THRESHOLD = 100 * 1024; // 100k
|
||||
|
||||
/**
|
||||
* Default compression type.
|
||||
*/
|
||||
|
@ -259,8 +265,10 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
for (String s : DEFAULT_VALUES.keySet()) {
|
||||
RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(s)));
|
||||
}
|
||||
RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION)));
|
||||
RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION_KEY)));
|
||||
RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION)));
|
||||
RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION_KEY)));
|
||||
RESERVED_KEYWORDS.add(new Bytes(IS_MOB_BYTES));
|
||||
RESERVED_KEYWORDS.add(new Bytes(MOB_THRESHOLD_BYTES));
|
||||
}
|
||||
|
||||
private static final int UNINITIALIZED = -1;
|
||||
|
@ -982,6 +990,10 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
/* TTL for now, we can add more as we neeed */
|
||||
if (key.equals(HColumnDescriptor.TTL)) {
|
||||
unit = Unit.TIME_INTERVAL;
|
||||
} else if (key.equals(HColumnDescriptor.MOB_THRESHOLD)) {
|
||||
unit = Unit.LONG;
|
||||
} else if (key.equals(HColumnDescriptor.IS_MOB)) {
|
||||
unit = Unit.BOOLEAN;
|
||||
} else {
|
||||
unit = Unit.NONE;
|
||||
}
|
||||
|
@ -1172,4 +1184,46 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
setValue(Bytes.toBytes(ENCRYPTION_KEY), keyBytes);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the mob threshold of the family.
|
||||
* If the size of a cell value is larger than this threshold, it's regarded as a mob.
|
||||
* The default threshold is 1024*100(100K)B.
|
||||
* @return The mob threshold.
|
||||
*/
|
||||
public long getMobThreshold() {
|
||||
byte[] threshold = getValue(MOB_THRESHOLD_BYTES);
|
||||
return threshold != null && threshold.length == Bytes.SIZEOF_LONG ? Bytes.toLong(threshold)
|
||||
: DEFAULT_MOB_THRESHOLD;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the mob threshold of the family.
|
||||
* @param threshold The mob threshold.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public HColumnDescriptor setMobThreshold(long threshold) {
|
||||
setValue(MOB_THRESHOLD_BYTES, Bytes.toBytes(threshold));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets whether the mob is enabled for the family.
|
||||
* @return True if the mob is enabled for the family.
|
||||
*/
|
||||
public boolean isMobEnabled() {
|
||||
byte[] isMobEnabled = getValue(IS_MOB_BYTES);
|
||||
return isMobEnabled != null && isMobEnabled.length == Bytes.SIZEOF_BOOLEAN
|
||||
&& Bytes.toBoolean(isMobEnabled);
|
||||
}
|
||||
|
||||
/**
|
||||
* Enables the mob for the family.
|
||||
* @param isMobEnabled Whether to enable the mob for the family.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public HColumnDescriptor setMobEnabled(boolean isMobEnabled) {
|
||||
setValue(IS_MOB_BYTES, Bytes.toBytes(isMobEnabled));
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1494,4 +1494,56 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @throws IOException
|
||||
*/
|
||||
public int getMasterInfoPort() throws IOException;
|
||||
|
||||
/**
|
||||
* Compact the mob files in all mob-enabled column families. Asynchronous operation.
|
||||
*
|
||||
* @param tableName table to compact
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
void compactMobs(final TableName tableName) throws IOException,
|
||||
InterruptedException;
|
||||
|
||||
/**
|
||||
* Compact the mob files in a mob-enabled column family. Asynchronous operation.
|
||||
*
|
||||
* @param tableName table to compact
|
||||
* @param columnFamily column family within a table
|
||||
* @throws IOException if not a mob column family or if a remote or network exception occurs
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
void compactMob(final TableName tableName, final byte[] columnFamily) throws IOException,
|
||||
InterruptedException;
|
||||
|
||||
/**
|
||||
* Major compact the mob files in all mob-enabled column family. Asynchronous operation.
|
||||
*
|
||||
* @param tableName table to compact
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
void majorCompactMobs(final TableName tableName) throws IOException,
|
||||
InterruptedException;
|
||||
|
||||
/**
|
||||
* Major compact the mob files in a mob-enabled column family. Asynchronous operation.
|
||||
*
|
||||
* @param tableName table to compact
|
||||
* @param columnFamily column family within a table
|
||||
* @throws IOException if not a mob column family or if a remote or network exception occurs
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
void majorCompactMob(final TableName tableName, final byte[] columnFamily) throws IOException,
|
||||
InterruptedException;
|
||||
|
||||
/**
|
||||
* Get the current compaction state of a table. It could be in a compaction, or none.
|
||||
*
|
||||
* @param tableName table to examine
|
||||
* @return the current compaction state
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
AdminProtos.GetRegionInfoResponse.CompactionState getMobCompactionState(final TableName tableName)
|
||||
throws IOException;
|
||||
}
|
||||
|
|
|
@ -4120,6 +4120,108 @@ public class HBaseAdmin implements Admin {
|
|||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void compactMob(final TableName tableName, final byte[] columnFamily)
|
||||
throws IOException, InterruptedException {
|
||||
checkTableNameNotNull(tableName);
|
||||
checkFamilyNameNotNull(columnFamily);
|
||||
validateMobColumnFamily(tableName, columnFamily);
|
||||
compactMob(tableName, columnFamily, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void compactMobs(final TableName tableName) throws IOException, InterruptedException {
|
||||
checkTableNameNotNull(tableName);
|
||||
compactMob(tableName, null, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void majorCompactMob(final TableName tableName, final byte[] columnFamily)
|
||||
throws IOException, InterruptedException {
|
||||
checkTableNameNotNull(tableName);
|
||||
checkFamilyNameNotNull(columnFamily);
|
||||
validateMobColumnFamily(tableName, columnFamily);
|
||||
compactMob(tableName, columnFamily, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void majorCompactMobs(final TableName tableName) throws IOException, InterruptedException {
|
||||
checkTableNameNotNull(tableName);
|
||||
compactMob(tableName, null, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public CompactionState getMobCompactionState(TableName tableName) throws IOException {
|
||||
checkTableNameNotNull(tableName);
|
||||
try {
|
||||
ServerName master = getClusterStatus().getMaster();
|
||||
HRegionInfo info = new HRegionInfo(tableName, Bytes.toBytes(".mob"),
|
||||
HConstants.EMPTY_END_ROW, false, 0);
|
||||
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
|
||||
info.getRegionName(), true);
|
||||
GetRegionInfoResponse response = this.connection.getAdmin(master)
|
||||
.getRegionInfo(null, request);
|
||||
return response.getCompactionState();
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts the mob files in a mob-enabled column family. Asynchronous operation.
|
||||
* @param tableName The table to compact.
|
||||
* @param columnFamily The column family to compact. If it is null, all the mob-enabled
|
||||
* column families in this table will be compacted.
|
||||
* @param major Whether to select all the mob files in the compaction.
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private void compactMob(final TableName tableName, final byte[] columnFamily, boolean major)
|
||||
throws IOException, InterruptedException {
|
||||
// get the mob region info, this is a dummy region.
|
||||
HRegionInfo info = new HRegionInfo(tableName, Bytes.toBytes(".mob"), HConstants.EMPTY_END_ROW,
|
||||
false, 0);
|
||||
ServerName master = getClusterStatus().getMaster();
|
||||
compact(master, info, major, columnFamily);
|
||||
}
|
||||
|
||||
private void checkTableNameNotNull(TableName tableName) {
|
||||
if (tableName == null) {
|
||||
throw new IllegalArgumentException("TableName cannot be null");
|
||||
}
|
||||
}
|
||||
|
||||
private void checkFamilyNameNotNull(byte[] columnFamily) {
|
||||
if (columnFamily == null) {
|
||||
throw new IllegalArgumentException("The column family name cannot be null");
|
||||
}
|
||||
}
|
||||
|
||||
private void validateMobColumnFamily(TableName tableName, byte[] columnFamily)
|
||||
throws IOException {
|
||||
HTableDescriptor htd = getTableDescriptor(tableName);
|
||||
HColumnDescriptor family = htd.getFamily(columnFamily);
|
||||
if (family == null || !family.isMobEnabled()) {
|
||||
throw new IllegalArgumentException("Column family " + columnFamily
|
||||
+ " is not a mob column family");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Future that waits on a procedure result.
|
||||
* Returned by the async version of the Admin calls,
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
|
@ -90,7 +91,7 @@ public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter {
|
|||
// Call may be null because it may have timedout and been cleaned up on this side already
|
||||
if (call.responseDefaultType != null) {
|
||||
Message.Builder builder = call.responseDefaultType.newBuilderForType();
|
||||
builder.mergeDelimitedFrom(in);
|
||||
ProtobufUtil.mergeDelimitedFrom(builder, in);
|
||||
value = builder.build();
|
||||
}
|
||||
CellScanner cellBlockScanner = null;
|
||||
|
|
|
@ -959,7 +959,7 @@ public class RpcClientImpl extends AbstractRpcClient {
|
|||
Message value = null;
|
||||
if (call.responseDefaultType != null) {
|
||||
Builder builder = call.responseDefaultType.newBuilderForType();
|
||||
builder.mergeDelimitedFrom(in);
|
||||
ProtobufUtil.mergeDelimitedFrom(builder, in);
|
||||
value = builder.build();
|
||||
}
|
||||
CellScanner cellBlockScanner = null;
|
||||
|
|
|
@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpeci
|
|||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
|
@ -67,6 +68,7 @@ import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
|||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.io.LimitInputStream;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
|
||||
|
@ -154,6 +156,7 @@ import com.google.common.collect.ArrayListMultimap;
|
|||
import com.google.common.collect.ListMultimap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.CodedInputStream;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Parser;
|
||||
|
@ -165,8 +168,8 @@ import com.google.protobuf.TextFormat;
|
|||
/**
|
||||
* Protobufs utility.
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED",
|
||||
justification="None. Address sometime.")
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
|
||||
value="DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED", justification="None. Address sometime.")
|
||||
@InterfaceAudience.Private // TODO: some clients (Hive, etc) use this class
|
||||
public final class ProtobufUtil {
|
||||
|
||||
|
@ -1585,7 +1588,7 @@ public final class ProtobufUtil {
|
|||
throws IOException {
|
||||
CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
|
||||
.setCall(call).setRegion(
|
||||
RequestConverter.buildRegionSpecifier(REGION_NAME, HConstants.EMPTY_BYTE_ARRAY)).build();
|
||||
RequestConverter.buildRegionSpecifier(REGION_NAME, HConstants.EMPTY_BYTE_ARRAY)).build();
|
||||
try {
|
||||
CoprocessorServiceResponse response =
|
||||
client.execMasterService(null, request);
|
||||
|
@ -1887,7 +1890,8 @@ public final class ProtobufUtil {
|
|||
if (proto.getType() != AccessControlProtos.Permission.Type.Global) {
|
||||
return toTablePermission(proto);
|
||||
} else {
|
||||
List<Permission.Action> actions = toPermissionActions(proto.getGlobalPermission().getActionList());
|
||||
List<Permission.Action> actions = toPermissionActions(
|
||||
proto.getGlobalPermission().getActionList());
|
||||
return new Permission(actions.toArray(new Permission.Action[actions.size()]));
|
||||
}
|
||||
}
|
||||
|
@ -1954,7 +1958,7 @@ public final class ProtobufUtil {
|
|||
AccessControlProtos.NamespacePermission.Builder builder =
|
||||
AccessControlProtos.NamespacePermission.newBuilder();
|
||||
builder.setNamespaceName(ByteString.copyFromUtf8(tablePerm.getNamespace()));
|
||||
Permission.Action actions[] = perm.getActions();
|
||||
Permission.Action[] actions = perm.getActions();
|
||||
if (actions != null) {
|
||||
for (Permission.Action a : actions) {
|
||||
builder.addAction(toPermissionAction(a));
|
||||
|
@ -2997,6 +3001,30 @@ public final class ProtobufUtil {
|
|||
return desc.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* This version of protobuf's mergeDelimitedFrom avoid the hard-coded 64MB limit for decoding
|
||||
* buffers
|
||||
* @param builder current message builder
|
||||
* @param in Inputsream with delimited protobuf data
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void mergeDelimitedFrom(Message.Builder builder, InputStream in)
|
||||
throws IOException {
|
||||
// This used to be builder.mergeDelimitedFrom(in);
|
||||
// but is replaced to allow us to bump the protobuf size limit.
|
||||
final int firstByte = in.read();
|
||||
if (firstByte == -1) {
|
||||
// bail out. (was return false;)
|
||||
} else {
|
||||
final int size = CodedInputStream.readRawVarint32(firstByte, in);
|
||||
final InputStream limitedInput = new LimitInputStream(in, size);
|
||||
final CodedInputStream codedInput = CodedInputStream.newInstance(limitedInput);
|
||||
codedInput.setSizeLimit(size);
|
||||
builder.mergeFrom(codedInput);
|
||||
codedInput.checkLastTagWas(0);
|
||||
}
|
||||
}
|
||||
|
||||
public static ReplicationLoadSink toReplicationLoadSink(
|
||||
ClusterStatusProtos.ReplicationLoadSink cls) {
|
||||
return new ReplicationLoadSink(cls.getAgeOfLastAppliedOp(), cls.getTimeStampsOfLastAppliedOp());
|
||||
|
|
|
@ -0,0 +1,16 @@
|
|||
--- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
|
||||
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
|
||||
@@ -124,12 +126,12 @@
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
|
||||
-import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaScope;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaType;
|
|
@ -25,6 +25,8 @@ import org.apache.hadoop.hbase.io.compress.Compression;
|
|||
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.PrettyPrinter;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.BuilderStyleTest;
|
||||
|
@ -58,6 +60,8 @@ public class TestHColumnDescriptor {
|
|||
hcd.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
|
||||
hcd.setBloomFilterType(BloomType.ROW);
|
||||
hcd.setCompressionType(Algorithm.SNAPPY);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(1000L);
|
||||
|
||||
|
||||
byte [] bytes = hcd.toByteArray();
|
||||
|
@ -74,6 +78,8 @@ public class TestHColumnDescriptor {
|
|||
assertTrue(deserializedHcd.getCompressionType().equals(Compression.Algorithm.SNAPPY));
|
||||
assertTrue(deserializedHcd.getDataBlockEncoding().equals(DataBlockEncoding.FAST_DIFF));
|
||||
assertTrue(deserializedHcd.getBloomFilterType().equals(BloomType.ROW));
|
||||
assertEquals(hcd.isMobEnabled(), deserializedHcd.isMobEnabled());
|
||||
assertEquals(hcd.getMobThreshold(), deserializedHcd.getMobThreshold());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -101,6 +107,18 @@ public class TestHColumnDescriptor {
|
|||
assertEquals(null, desc.getConfigurationValue(key));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMobValuesInHColumnDescriptorShouldReadable() {
|
||||
boolean isMob = true;
|
||||
long threshold = 1000;
|
||||
String isMobString = PrettyPrinter.format(Bytes.toStringBinary(Bytes.toBytes(isMob)),
|
||||
HColumnDescriptor.getUnit(HColumnDescriptor.IS_MOB));
|
||||
String thresholdString = PrettyPrinter.format(Bytes.toStringBinary(Bytes.toBytes(threshold)),
|
||||
HColumnDescriptor.getUnit(HColumnDescriptor.MOB_THRESHOLD));
|
||||
assertEquals(String.valueOf(isMob), isMobString);
|
||||
assertEquals(String.valueOf(threshold), thresholdString);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClassMethodsAreBuilderStyle() {
|
||||
/* HColumnDescriptor should have a builder style setup where setXXX/addXXX methods
|
||||
|
|
|
@ -28,6 +28,11 @@ public final class TagType {
|
|||
public static final byte VISIBILITY_TAG_TYPE = (byte) 2;
|
||||
// public static final byte LOG_REPLAY_TAG_TYPE = (byte) 3; // deprecated
|
||||
public static final byte VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE = (byte)4;
|
||||
|
||||
// mob tags
|
||||
public static final byte MOB_REFERENCE_TAG_TYPE = (byte) 5;
|
||||
public static final byte MOB_TABLE_NAME_TAG_TYPE = (byte) 6;
|
||||
|
||||
// String based tag type used in replication
|
||||
public static final byte STRING_VIS_TAG_TYPE = (byte) 7;
|
||||
public static final byte TTL_TAG_TYPE = (byte)8;
|
||||
|
|
|
@ -27,6 +27,8 @@ public class PrettyPrinter {
|
|||
|
||||
public enum Unit {
|
||||
TIME_INTERVAL,
|
||||
LONG,
|
||||
BOOLEAN,
|
||||
NONE
|
||||
}
|
||||
|
||||
|
@ -36,6 +38,14 @@ public class PrettyPrinter {
|
|||
case TIME_INTERVAL:
|
||||
human.append(humanReadableTTL(Long.parseLong(value)));
|
||||
break;
|
||||
case LONG:
|
||||
byte[] longBytes = Bytes.toBytesBinary(value);
|
||||
human.append(String.valueOf(Bytes.toLong(longBytes)));
|
||||
break;
|
||||
case BOOLEAN:
|
||||
byte[] booleanBytes = Bytes.toBytesBinary(value);
|
||||
human.append(String.valueOf(Bytes.toBoolean(booleanBytes)));
|
||||
break;
|
||||
default:
|
||||
human.append(value);
|
||||
}
|
||||
|
|
|
@ -1553,4 +1553,121 @@ possible configurations would overwhelm and obscure the important.
|
|||
0.x Abort only when this percent of handlers have died;
|
||||
1 Abort only all of the handers have died.</description>
|
||||
</property>
|
||||
<!-- Mob properties. -->
|
||||
<property>
|
||||
<name>hbase.mob.file.cache.size</name>
|
||||
<value>1000</value>
|
||||
<description>
|
||||
Number of opened file handlers to cache.
|
||||
A larger value will benefit reads by providing more file handlers per mob
|
||||
file cache and would reduce frequent file opening and closing.
|
||||
However, if this is set too high, this could lead to a "too many opened file handlers"
|
||||
The default value is 1000.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.cache.evict.period</name>
|
||||
<value>3600</value>
|
||||
<description>
|
||||
The amount of time in seconds before the mob cache evicts cached mob files.
|
||||
The default value is 3600 seconds.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.cache.evict.remain.ratio</name>
|
||||
<value>0.5f</value>
|
||||
<description>
|
||||
The ratio (between 0.0 and 1.0) of files that remains cached after an eviction
|
||||
is triggered when the number of cached mob files exceeds the hbase.mob.file.cache.size.
|
||||
The default value is 0.5f.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.sweep.tool.compaction.ratio</name>
|
||||
<value>0.5f</value>
|
||||
<description>
|
||||
If there're too many cells deleted in a mob file, it's regarded
|
||||
as an invalid file and needs to be merged.
|
||||
If existingCellsSize/mobFileSize is less than ratio, it's regarded
|
||||
as an invalid file. The default value is 0.5f.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.sweep.tool.compaction.mergeable.size</name>
|
||||
<value>134217728</value>
|
||||
<description>
|
||||
If the size of a mob file is less than this value, it's regarded as a small
|
||||
file and needs to be merged. The default value is 128MB.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.sweep.tool.compaction.memstore.flush.size</name>
|
||||
<value>134217728</value>
|
||||
<description>
|
||||
The flush size for the memstore used by sweep job. Each sweep reducer owns such a memstore.
|
||||
The default value is 128MB.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.master.mob.ttl.cleaner.period</name>
|
||||
<value>86400</value>
|
||||
<description>
|
||||
The period that ExpiredMobFileCleanerChore runs. The unit is second.
|
||||
The default value is one day.
|
||||
The MOB file name uses only the date part of the file creation time in it. We use this
|
||||
time for deciding TTL expiry of the files. So the removal of TTL expired files might be
|
||||
delayed. The max delay might be 24 hrs.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.compaction.mergeable.threshold</name>
|
||||
<value>201326592</value>
|
||||
<description>
|
||||
If the size of a mob file is less than this value, it's regarded as a small
|
||||
file and needs to be merged in mob compaction. The default value is 192MB.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.delfile.max.count</name>
|
||||
<value>3</value>
|
||||
<description>
|
||||
The max number of del files that is allowed in the mob compaction.
|
||||
In the mob compaction, when the number of existing del files is larger than
|
||||
this value, they are merged until number of del files is not larger this value.
|
||||
The default value is 3.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.compaction.batch.size</name>
|
||||
<value>100</value>
|
||||
<description>
|
||||
The max number of the mob files that is allowed in a batch of the mob compaction.
|
||||
The mob compaction merges the small mob files to bigger ones. If the number of the
|
||||
small files is very large, it could lead to a "too many opened file handlers" in the merge.
|
||||
And the merge has to be split into batches. This value limits the number of mob files
|
||||
that are selected in a batch of the mob compaction. The default value is 100.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.compaction.chore.period</name>
|
||||
<value>604800</value>
|
||||
<description>
|
||||
The period that MobCompactionChore runs. The unit is second.
|
||||
The default value is one week.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.compactor.class</name>
|
||||
<value>org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor</value>
|
||||
<description>
|
||||
Implementation of mob compactor, the default one is PartitionedMobCompactor.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.mob.compaction.threads.max</name>
|
||||
<value>1</value>
|
||||
<description>
|
||||
The max number of threads used in MobCompactor.
|
||||
</description>
|
||||
</property>
|
||||
</configuration>
|
||||
|
|
|
@ -267,6 +267,38 @@ public interface MetricsRegionServerSource extends BaseSource {
|
|||
String MAJOR_COMPACTED_CELLS_SIZE = "majorCompactedCellsSize";
|
||||
String MAJOR_COMPACTED_CELLS_SIZE_DESC =
|
||||
"The total amount of data processed during major compactions, in bytes";
|
||||
String CELLS_COUNT_COMPACTED_TO_MOB = "cellsCountCompactedToMob";
|
||||
String CELLS_COUNT_COMPACTED_TO_MOB_DESC =
|
||||
"The number of cells moved to mob during compaction";
|
||||
String CELLS_COUNT_COMPACTED_FROM_MOB = "cellsCountCompactedFromMob";
|
||||
String CELLS_COUNT_COMPACTED_FROM_MOB_DESC =
|
||||
"The number of cells moved from mob during compaction";
|
||||
String CELLS_SIZE_COMPACTED_TO_MOB = "cellsSizeCompactedToMob";
|
||||
String CELLS_SIZE_COMPACTED_TO_MOB_DESC =
|
||||
"The total amount of cells move to mob during compaction, in bytes";
|
||||
String CELLS_SIZE_COMPACTED_FROM_MOB = "cellsSizeCompactedFromMob";
|
||||
String CELLS_SIZE_COMPACTED_FROM_MOB_DESC =
|
||||
"The total amount of cells move from mob during compaction, in bytes";
|
||||
String MOB_FLUSH_COUNT = "mobFlushCount";
|
||||
String MOB_FLUSH_COUNT_DESC = "The number of the flushes in mob-enabled stores";
|
||||
String MOB_FLUSHED_CELLS_COUNT = "mobFlushedCellsCount";
|
||||
String MOB_FLUSHED_CELLS_COUNT_DESC = "The number of mob cells flushed to disk";
|
||||
String MOB_FLUSHED_CELLS_SIZE = "mobFlushedCellsSize";
|
||||
String MOB_FLUSHED_CELLS_SIZE_DESC = "The total amount of mob cells flushed to disk, in bytes";
|
||||
String MOB_SCAN_CELLS_COUNT = "mobScanCellsCount";
|
||||
String MOB_SCAN_CELLS_COUNT_DESC = "The number of scanned mob cells";
|
||||
String MOB_SCAN_CELLS_SIZE = "mobScanCellsSize";
|
||||
String MOB_SCAN_CELLS_SIZE_DESC = "The total amount of scanned mob cells, in bytes";
|
||||
String MOB_FILE_CACHE_ACCESS_COUNT = "mobFileCacheAccessCount";
|
||||
String MOB_FILE_CACHE_ACCESS_COUNT_DESC = "The count of accesses to the mob file cache";
|
||||
String MOB_FILE_CACHE_MISS_COUNT = "mobFileCacheMissCount";
|
||||
String MOB_FILE_CACHE_MISS_COUNT_DESC = "The count of misses to the mob file cache";
|
||||
String MOB_FILE_CACHE_HIT_PERCENT = "mobFileCacheHitPercent";
|
||||
String MOB_FILE_CACHE_HIT_PERCENT_DESC = "The hit percent to the mob file cache";
|
||||
String MOB_FILE_CACHE_EVICTED_COUNT = "mobFileCacheEvictedCount";
|
||||
String MOB_FILE_CACHE_EVICTED_COUNT_DESC = "The number of items evicted from the mob file cache";
|
||||
String MOB_FILE_CACHE_COUNT = "mobFileCacheCount";
|
||||
String MOB_FILE_CACHE_COUNT_DESC = "The count of cached mob files";
|
||||
|
||||
String HEDGED_READS = "hedgedReads";
|
||||
String HEDGED_READS_DESC = "The number of times we started a hedged read";
|
||||
|
|
|
@ -257,6 +257,76 @@ public interface MetricsRegionServerWrapper {
|
|||
*/
|
||||
long getMajorCompactedCellsSize();
|
||||
|
||||
/**
|
||||
* Gets the number of cells moved to mob during compaction.
|
||||
*/
|
||||
long getCellsCountCompactedToMob();
|
||||
|
||||
/**
|
||||
* Gets the number of cells moved from mob during compaction.
|
||||
*/
|
||||
long getCellsCountCompactedFromMob();
|
||||
|
||||
/**
|
||||
* Gets the total amount of cells moved to mob during compaction, in bytes.
|
||||
*/
|
||||
long getCellsSizeCompactedToMob();
|
||||
|
||||
/**
|
||||
* Gets the total amount of cells moved from mob during compaction, in bytes.
|
||||
*/
|
||||
long getCellsSizeCompactedFromMob();
|
||||
|
||||
/**
|
||||
* Gets the number of the flushes in mob-enabled stores.
|
||||
*/
|
||||
long getMobFlushCount();
|
||||
|
||||
/**
|
||||
* Gets the number of mob cells flushed to disk.
|
||||
*/
|
||||
long getMobFlushedCellsCount();
|
||||
|
||||
/**
|
||||
* Gets the total amount of mob cells flushed to disk, in bytes.
|
||||
*/
|
||||
long getMobFlushedCellsSize();
|
||||
|
||||
/**
|
||||
* Gets the number of scanned mob cells.
|
||||
*/
|
||||
long getMobScanCellsCount();
|
||||
|
||||
/**
|
||||
* Gets the total amount of scanned mob cells, in bytes.
|
||||
*/
|
||||
long getMobScanCellsSize();
|
||||
|
||||
/**
|
||||
* Gets the count of accesses to the mob file cache.
|
||||
*/
|
||||
long getMobFileCacheAccessCount();
|
||||
|
||||
/**
|
||||
* Gets the count of misses to the mob file cache.
|
||||
*/
|
||||
long getMobFileCacheMissCount();
|
||||
|
||||
/**
|
||||
* Gets the number of items evicted from the mob file cache.
|
||||
*/
|
||||
long getMobFileCacheEvictedCount();
|
||||
|
||||
/**
|
||||
* Gets the count of cached mob files.
|
||||
*/
|
||||
long getMobFileCacheCount();
|
||||
|
||||
/**
|
||||
* Gets the hit percent to the mob file cache.
|
||||
*/
|
||||
int getMobFileCacheHitPercent();
|
||||
|
||||
/**
|
||||
* @return Count of hedged read operations
|
||||
*/
|
||||
|
|
|
@ -259,6 +259,38 @@ public class MetricsRegionServerSourceImpl
|
|||
.addCounter(Interns.info(MAJOR_COMPACTED_CELLS_SIZE, MAJOR_COMPACTED_CELLS_SIZE_DESC),
|
||||
rsWrap.getMajorCompactedCellsSize())
|
||||
|
||||
.addCounter(
|
||||
Interns.info(CELLS_COUNT_COMPACTED_FROM_MOB, CELLS_COUNT_COMPACTED_FROM_MOB_DESC),
|
||||
rsWrap.getCellsCountCompactedFromMob())
|
||||
.addCounter(Interns.info(CELLS_COUNT_COMPACTED_TO_MOB, CELLS_COUNT_COMPACTED_TO_MOB_DESC),
|
||||
rsWrap.getCellsCountCompactedToMob())
|
||||
.addCounter(
|
||||
Interns.info(CELLS_SIZE_COMPACTED_FROM_MOB, CELLS_SIZE_COMPACTED_FROM_MOB_DESC),
|
||||
rsWrap.getCellsSizeCompactedFromMob())
|
||||
.addCounter(Interns.info(CELLS_SIZE_COMPACTED_TO_MOB, CELLS_SIZE_COMPACTED_TO_MOB_DESC),
|
||||
rsWrap.getCellsSizeCompactedToMob())
|
||||
.addCounter(Interns.info(MOB_FLUSH_COUNT, MOB_FLUSH_COUNT_DESC),
|
||||
rsWrap.getMobFlushCount())
|
||||
.addCounter(Interns.info(MOB_FLUSHED_CELLS_COUNT, MOB_FLUSHED_CELLS_COUNT_DESC),
|
||||
rsWrap.getMobFlushedCellsCount())
|
||||
.addCounter(Interns.info(MOB_FLUSHED_CELLS_SIZE, MOB_FLUSHED_CELLS_SIZE_DESC),
|
||||
rsWrap.getMobFlushedCellsSize())
|
||||
.addCounter(Interns.info(MOB_SCAN_CELLS_COUNT, MOB_SCAN_CELLS_COUNT_DESC),
|
||||
rsWrap.getMobScanCellsCount())
|
||||
.addCounter(Interns.info(MOB_SCAN_CELLS_SIZE, MOB_SCAN_CELLS_SIZE_DESC),
|
||||
rsWrap.getMobScanCellsSize())
|
||||
.addGauge(Interns.info(MOB_FILE_CACHE_COUNT, MOB_FILE_CACHE_COUNT_DESC),
|
||||
rsWrap.getMobFileCacheCount())
|
||||
.addCounter(Interns.info(MOB_FILE_CACHE_ACCESS_COUNT, MOB_FILE_CACHE_ACCESS_COUNT_DESC),
|
||||
rsWrap.getMobFileCacheAccessCount())
|
||||
.addCounter(Interns.info(MOB_FILE_CACHE_MISS_COUNT, MOB_FILE_CACHE_MISS_COUNT_DESC),
|
||||
rsWrap.getMobFileCacheMissCount())
|
||||
.addCounter(
|
||||
Interns.info(MOB_FILE_CACHE_EVICTED_COUNT, MOB_FILE_CACHE_EVICTED_COUNT_DESC),
|
||||
rsWrap.getMobFileCacheEvictedCount())
|
||||
.addGauge(Interns.info(MOB_FILE_CACHE_HIT_PERCENT, MOB_FILE_CACHE_HIT_PERCENT_DESC),
|
||||
rsWrap.getMobFileCacheHitPercent())
|
||||
|
||||
.addCounter(Interns.info(HEDGED_READS, HEDGED_READS_DESC), rsWrap.getHedgedReadOps())
|
||||
.addCounter(Interns.info(HEDGED_READ_WINS, HEDGED_READ_WINS_DESC),
|
||||
rsWrap.getHedgedReadWins())
|
||||
|
|
|
@ -55,7 +55,8 @@ public class IntegrationTestAcidGuarantees extends IntegrationTestBase {
|
|||
int numGetters = c.getInt("numGetters", 2);
|
||||
int numScanners = c.getInt("numScanners", 2);
|
||||
int numUniqueRows = c.getInt("numUniqueRows", 3);
|
||||
tag.runTestAtomicity(millis, numWriters, numGetters, numScanners, numUniqueRows, true);
|
||||
boolean useMob = c.getBoolean("useMob",false);
|
||||
tag.runTestAtomicity(millis, numWriters, numGetters, numScanners, numUniqueRows, true, useMob);
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,156 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.LoadTestDataGeneratorWithMOB;
|
||||
import org.apache.hadoop.hbase.util.LoadTestTool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Integration Test for MOB ingest.
|
||||
*/
|
||||
@Category(IntegrationTests.class)
|
||||
public class IntegrationTestIngestWithMOB extends IntegrationTestIngest {
|
||||
private static final char COLON = ':';
|
||||
|
||||
private byte[] mobColumnFamily = LoadTestTool.DEFAULT_COLUMN_FAMILY;
|
||||
public static final String THRESHOLD = "threshold";
|
||||
public static final String MIN_MOB_DATA_SIZE = "minMobDataSize";
|
||||
public static final String MAX_MOB_DATA_SIZE = "maxMobDataSize";
|
||||
private int threshold = 1024; // 1KB
|
||||
private int minMobDataSize = 512; // 512B
|
||||
private int maxMobDataSize = threshold * 5; // 5KB
|
||||
private static final long JUNIT_RUN_TIME = 2 * 60 * 1000; // 2 minutes
|
||||
|
||||
//similar to LOAD_TEST_TOOL_INIT_ARGS except OPT_IN_MEMORY is removed
|
||||
protected String[] LOAD_TEST_TOOL_MOB_INIT_ARGS = {
|
||||
LoadTestTool.OPT_COMPRESSION,
|
||||
LoadTestTool.OPT_DATA_BLOCK_ENCODING,
|
||||
LoadTestTool.OPT_ENCRYPTION,
|
||||
LoadTestTool.OPT_NUM_REGIONS_PER_SERVER,
|
||||
LoadTestTool.OPT_REGION_REPLICATION,
|
||||
};
|
||||
|
||||
@Override
|
||||
protected String[] getArgsForLoadTestToolInitTable() {
|
||||
List<String> args = new ArrayList<String>();
|
||||
args.add("-tn");
|
||||
args.add(getTablename().getNameAsString());
|
||||
// pass all remaining args from conf with keys <test class name>.<load test tool arg>
|
||||
String clazz = this.getClass().getSimpleName();
|
||||
for (String arg : LOAD_TEST_TOOL_MOB_INIT_ARGS) {
|
||||
String val = conf.get(String.format("%s.%s", clazz, arg));
|
||||
if (val != null) {
|
||||
args.add("-" + arg);
|
||||
args.add(val);
|
||||
}
|
||||
}
|
||||
args.add("-init_only");
|
||||
return args.toArray(new String[args.size()]);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void addOptions() {
|
||||
super.addOptions();
|
||||
super.addOptWithArg(THRESHOLD, "The threshold to classify cells to mob data");
|
||||
super.addOptWithArg(MIN_MOB_DATA_SIZE, "Minimum value size for mob data");
|
||||
super.addOptWithArg(MAX_MOB_DATA_SIZE, "Maximum value size for mob data");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processOptions(CommandLine cmd) {
|
||||
super.processOptions(cmd);
|
||||
if (cmd.hasOption(THRESHOLD)) {
|
||||
threshold = Integer.parseInt(cmd.getOptionValue(THRESHOLD));
|
||||
}
|
||||
if (cmd.hasOption(MIN_MOB_DATA_SIZE)) {
|
||||
minMobDataSize = Integer.parseInt(cmd.getOptionValue(MIN_MOB_DATA_SIZE));
|
||||
}
|
||||
if (cmd.hasOption(MAX_MOB_DATA_SIZE)) {
|
||||
maxMobDataSize = Integer.parseInt(cmd.getOptionValue(MAX_MOB_DATA_SIZE));
|
||||
}
|
||||
if (minMobDataSize > maxMobDataSize) {
|
||||
throw new IllegalArgumentException(
|
||||
"The minMobDataSize should not be larger than minMobDataSize");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIngest() throws Exception {
|
||||
runIngestTest(JUNIT_RUN_TIME, 100, 10, 1024, 10, 20);
|
||||
};
|
||||
|
||||
@Override
|
||||
protected void initTable() throws IOException {
|
||||
super.initTable();
|
||||
|
||||
TableName tableName = getTablename();
|
||||
Admin admin = ConnectionFactory.createConnection().getAdmin();
|
||||
HTableDescriptor tableDesc = admin.getTableDescriptor(tableName);
|
||||
LOG.info("Disabling table " + getTablename());
|
||||
admin.disableTable(tableName);
|
||||
for (HColumnDescriptor columnDescriptor : tableDesc.getFamilies()) {
|
||||
if(Arrays.equals(columnDescriptor.getName(), mobColumnFamily)) {
|
||||
columnDescriptor.setMobEnabled(true);
|
||||
columnDescriptor.setMobThreshold((long) threshold);
|
||||
admin.modifyColumn(tableName, columnDescriptor);
|
||||
}
|
||||
}
|
||||
LOG.info("Enabling table " + getTablename());
|
||||
admin.enableTable(tableName);
|
||||
admin.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String[] getArgsForLoadTestTool(String mode, String modeSpecificArg, long startKey,
|
||||
long numKeys) {
|
||||
String[] args = super.getArgsForLoadTestTool(mode, modeSpecificArg, startKey, numKeys);
|
||||
List<String> tmp = new ArrayList<String>(Arrays.asList(args));
|
||||
// LoadTestDataGeneratorMOB:mobColumnFamily:minMobDataSize:maxMobDataSize
|
||||
tmp.add(HIPHEN + LoadTestTool.OPT_GENERATOR);
|
||||
StringBuilder sb = new StringBuilder(LoadTestDataGeneratorWithMOB.class.getName());
|
||||
sb.append(COLON);
|
||||
sb.append(Bytes.toString(mobColumnFamily));
|
||||
sb.append(COLON);
|
||||
sb.append(minMobDataSize);
|
||||
sb.append(COLON);
|
||||
sb.append(maxMobDataSize);
|
||||
tmp.add(sb.toString());
|
||||
return tmp.toArray(new String[tmp.size()]);
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
IntegrationTestingUtility.setUseDistributedCluster(conf);
|
||||
int ret = ToolRunner.run(conf, new IntegrationTestIngestWithMOB(), args);
|
||||
System.exit(ret);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/**
|
||||
* 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.chaos.actions;
|
||||
|
||||
import org.apache.commons.lang.math.RandomUtils;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
|
||||
/**
|
||||
* Action that queues a table compaction.
|
||||
*/
|
||||
public class CompactMobAction extends Action {
|
||||
private final TableName tableName;
|
||||
private final int majorRatio;
|
||||
private final long sleepTime;
|
||||
|
||||
public CompactMobAction(TableName tableName, float majorRatio) {
|
||||
this(-1, tableName, majorRatio);
|
||||
}
|
||||
|
||||
public CompactMobAction(
|
||||
int sleepTime, TableName tableName, float majorRatio) {
|
||||
this.tableName = tableName;
|
||||
this.majorRatio = (int) (100 * majorRatio);
|
||||
this.sleepTime = sleepTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void perform() throws Exception {
|
||||
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
|
||||
Admin admin = util.getHBaseAdmin();
|
||||
boolean major = RandomUtils.nextInt(100) < majorRatio;
|
||||
|
||||
LOG.info("Performing action: Compact mob of table " + tableName + ", major=" + major);
|
||||
try {
|
||||
if (major) {
|
||||
admin.majorCompactMobs(tableName);
|
||||
} else {
|
||||
admin.compactMobs(tableName);
|
||||
}
|
||||
} catch (Exception ex) {
|
||||
LOG.warn("Mob Compaction failed, might be caused by other chaos: " + ex.getMessage());
|
||||
}
|
||||
if (sleepTime > 0) {
|
||||
Thread.sleep(sleepTime);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,67 @@
|
|||
/**
|
||||
* 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.chaos.factories;
|
||||
|
||||
import org.apache.hadoop.hbase.chaos.actions.*;
|
||||
import org.apache.hadoop.hbase.chaos.monkies.ChaosMonkey;
|
||||
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
|
||||
import org.apache.hadoop.hbase.chaos.policies.PeriodicRandomActionPolicy;
|
||||
import org.apache.hadoop.hbase.chaos.policies.TwoConcurrentActionPolicy;
|
||||
|
||||
/**
|
||||
* This is a copy of NoKillMonkeyFactory that also does mob compactions.
|
||||
*/
|
||||
public class MobNoKillMonkeyFactory extends MonkeyFactory {
|
||||
@Override public ChaosMonkey build() {
|
||||
Action[] actions1 = new Action[] {
|
||||
new CompactMobAction(tableName, MonkeyConstants.DEFAULT_PERIODIC_ACTION1_PERIOD),
|
||||
new CompactTableAction(tableName, MonkeyConstants.DEFAULT_PERIODIC_ACTION1_PERIOD),
|
||||
new CompactRandomRegionOfTableAction(tableName,
|
||||
MonkeyConstants.DEFAULT_COMPACT_RANDOM_REGION_RATIO),
|
||||
new FlushTableAction(tableName),
|
||||
new FlushRandomRegionOfTableAction(tableName),
|
||||
new MoveRandomRegionOfTableAction(tableName)
|
||||
};
|
||||
|
||||
Action[] actions2 = new Action[] {
|
||||
new SplitRandomRegionOfTableAction(tableName),
|
||||
new MergeRandomAdjacentRegionsOfTableAction(tableName),
|
||||
new SnapshotTableAction(tableName),
|
||||
new AddColumnAction(tableName),
|
||||
new RemoveColumnAction(tableName, columnFamilies),
|
||||
new ChangeEncodingAction(tableName),
|
||||
new ChangeCompressionAction(tableName),
|
||||
new ChangeBloomFilterAction(tableName),
|
||||
new ChangeVersionsAction(tableName)
|
||||
};
|
||||
|
||||
Action[] actions3 = new Action[] {
|
||||
new MoveRegionsOfTableAction(MonkeyConstants.DEFAULT_MOVE_REGIONS_SLEEP_TIME,
|
||||
MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, tableName),
|
||||
new MoveRandomRegionOfTableAction(MonkeyConstants.DEFAULT_RESTART_ACTIVE_MASTER_SLEEP_TIME,
|
||||
tableName), };
|
||||
|
||||
Action[] actions4 = new Action[] { new DumpClusterStatusAction() };
|
||||
|
||||
return new PolicyBasedChaosMonkey(util,
|
||||
new TwoConcurrentActionPolicy(MonkeyConstants.DEFAULT_PERIODIC_ACTION1_PERIOD, actions1,
|
||||
actions2),
|
||||
new PeriodicRandomActionPolicy(MonkeyConstants.DEFAULT_PERIODIC_ACTION2_PERIOD,actions3),
|
||||
new PeriodicRandomActionPolicy(MonkeyConstants.DEFAULT_PERIODIC_ACTION4_PERIOD,actions4));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,156 @@
|
|||
/**
|
||||
* 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.chaos.factories;
|
||||
|
||||
import org.apache.hadoop.hbase.chaos.actions.*;
|
||||
import org.apache.hadoop.hbase.chaos.monkies.ChaosMonkey;
|
||||
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
|
||||
import org.apache.hadoop.hbase.chaos.policies.CompositeSequentialPolicy;
|
||||
import org.apache.hadoop.hbase.chaos.policies.DoActionsOncePolicy;
|
||||
import org.apache.hadoop.hbase.chaos.policies.PeriodicRandomActionPolicy;
|
||||
|
||||
/**
|
||||
* This is a copy of SlowDeterministicMonkeyFactory that also does mob compactions.
|
||||
*/
|
||||
public class MobSlowDeterministicMonkeyFactory extends MonkeyFactory {
|
||||
|
||||
private long action1Period;
|
||||
private long action2Period;
|
||||
private long action3Period;
|
||||
private long action4Period;
|
||||
private long moveRegionsMaxTime;
|
||||
private long moveRegionsSleepTime;
|
||||
private long moveRandomRegionSleepTime;
|
||||
private long restartRandomRSSleepTime;
|
||||
private long batchRestartRSSleepTime;
|
||||
private float batchRestartRSRatio;
|
||||
private long restartActiveMasterSleepTime;
|
||||
private long rollingBatchRestartRSSleepTime;
|
||||
private float rollingBatchRestartRSRatio;
|
||||
private long restartRsHoldingMetaSleepTime;
|
||||
private float compactTableRatio;
|
||||
private float compactRandomRegionRatio;
|
||||
|
||||
@Override
|
||||
public ChaosMonkey build() {
|
||||
|
||||
loadProperties();
|
||||
// Actions such as compact/flush a table/region,
|
||||
// move one region around. They are not so destructive,
|
||||
// can be executed more frequently.
|
||||
Action[] actions1 = new Action[] {
|
||||
new CompactMobAction(tableName, compactTableRatio),
|
||||
new CompactTableAction(tableName, compactTableRatio),
|
||||
new CompactRandomRegionOfTableAction(tableName, compactRandomRegionRatio),
|
||||
new FlushTableAction(tableName),
|
||||
new FlushRandomRegionOfTableAction(tableName),
|
||||
new MoveRandomRegionOfTableAction(tableName)
|
||||
};
|
||||
|
||||
// Actions such as split/merge/snapshot.
|
||||
// They should not cause data loss, or unreliability
|
||||
// such as region stuck in transition.
|
||||
Action[] actions2 = new Action[] {
|
||||
new SplitRandomRegionOfTableAction(tableName),
|
||||
new MergeRandomAdjacentRegionsOfTableAction(tableName),
|
||||
new SnapshotTableAction(tableName),
|
||||
new AddColumnAction(tableName),
|
||||
new RemoveColumnAction(tableName, columnFamilies),
|
||||
new ChangeEncodingAction(tableName),
|
||||
new ChangeCompressionAction(tableName),
|
||||
new ChangeBloomFilterAction(tableName),
|
||||
new ChangeVersionsAction(tableName)
|
||||
};
|
||||
|
||||
// Destructive actions to mess things around.
|
||||
Action[] actions3 = new Action[] {
|
||||
new MoveRegionsOfTableAction(moveRegionsSleepTime, moveRegionsMaxTime,
|
||||
tableName),
|
||||
new MoveRandomRegionOfTableAction(moveRandomRegionSleepTime, tableName),
|
||||
new RestartRandomRsAction(restartRandomRSSleepTime),
|
||||
new BatchRestartRsAction(batchRestartRSSleepTime, batchRestartRSRatio),
|
||||
new RestartActiveMasterAction(restartActiveMasterSleepTime),
|
||||
new RollingBatchRestartRsAction(rollingBatchRestartRSSleepTime,
|
||||
rollingBatchRestartRSRatio),
|
||||
new RestartRsHoldingMetaAction(restartRsHoldingMetaSleepTime)
|
||||
};
|
||||
|
||||
// Action to log more info for debugging
|
||||
Action[] actions4 = new Action[] {
|
||||
new DumpClusterStatusAction()
|
||||
};
|
||||
|
||||
return new PolicyBasedChaosMonkey(util,
|
||||
new PeriodicRandomActionPolicy(action1Period, actions1),
|
||||
new PeriodicRandomActionPolicy(action2Period, actions2),
|
||||
new CompositeSequentialPolicy(
|
||||
new DoActionsOncePolicy(action3Period, actions3),
|
||||
new PeriodicRandomActionPolicy(action3Period, actions3)),
|
||||
new PeriodicRandomActionPolicy(action4Period, actions4));
|
||||
}
|
||||
|
||||
private void loadProperties() {
|
||||
|
||||
action1Period = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.PERIODIC_ACTION1_PERIOD,
|
||||
MonkeyConstants.DEFAULT_PERIODIC_ACTION1_PERIOD + ""));
|
||||
action2Period = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.PERIODIC_ACTION2_PERIOD,
|
||||
MonkeyConstants.DEFAULT_PERIODIC_ACTION2_PERIOD + ""));
|
||||
action3Period = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.COMPOSITE_ACTION3_PERIOD,
|
||||
MonkeyConstants.DEFAULT_COMPOSITE_ACTION3_PERIOD + ""));
|
||||
action4Period = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.PERIODIC_ACTION4_PERIOD,
|
||||
MonkeyConstants.DEFAULT_PERIODIC_ACTION4_PERIOD + ""));
|
||||
moveRegionsMaxTime = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.MOVE_REGIONS_MAX_TIME,
|
||||
MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME + ""));
|
||||
moveRegionsSleepTime = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.MOVE_REGIONS_SLEEP_TIME,
|
||||
MonkeyConstants.DEFAULT_MOVE_REGIONS_SLEEP_TIME + ""));
|
||||
moveRandomRegionSleepTime = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.MOVE_RANDOM_REGION_SLEEP_TIME,
|
||||
MonkeyConstants.DEFAULT_MOVE_RANDOM_REGION_SLEEP_TIME + ""));
|
||||
restartRandomRSSleepTime = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.RESTART_RANDOM_RS_SLEEP_TIME,
|
||||
MonkeyConstants.DEFAULT_RESTART_RANDOM_RS_SLEEP_TIME + ""));
|
||||
batchRestartRSSleepTime = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.BATCH_RESTART_RS_SLEEP_TIME,
|
||||
MonkeyConstants.DEFAULT_BATCH_RESTART_RS_SLEEP_TIME + ""));
|
||||
restartActiveMasterSleepTime = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.RESTART_ACTIVE_MASTER_SLEEP_TIME,
|
||||
MonkeyConstants.DEFAULT_RESTART_ACTIVE_MASTER_SLEEP_TIME + ""));
|
||||
rollingBatchRestartRSSleepTime = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.ROLLING_BATCH_RESTART_RS_SLEEP_TIME,
|
||||
MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_SLEEP_TIME + ""));
|
||||
rollingBatchRestartRSRatio = Float.parseFloat(this.properties.getProperty(
|
||||
MonkeyConstants.ROLLING_BATCH_RESTART_RS_RATIO,
|
||||
MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_RATIO + ""));
|
||||
restartRsHoldingMetaSleepTime = Long.parseLong(this.properties.getProperty(
|
||||
MonkeyConstants.RESTART_RS_HOLDING_META_SLEEP_TIME,
|
||||
MonkeyConstants.DEFAULT_RESTART_RS_HOLDING_META_SLEEP_TIME + ""));
|
||||
compactTableRatio = Float.parseFloat(this.properties.getProperty(
|
||||
MonkeyConstants.COMPACT_TABLE_ACTION_RATIO,
|
||||
MonkeyConstants.DEFAULT_COMPACT_TABLE_ACTION_RATIO + ""));
|
||||
compactRandomRegionRatio = Float.parseFloat(this.properties.getProperty(
|
||||
MonkeyConstants.COMPACT_RANDOM_REGION_RATIO,
|
||||
MonkeyConstants.DEFAULT_COMPACT_RANDOM_REGION_RATIO + ""));
|
||||
}
|
||||
}
|
|
@ -70,6 +70,8 @@ public abstract class MonkeyFactory {
|
|||
public static final String STRESS_AM = "stressAM";
|
||||
public static final String NO_KILL = "noKill";
|
||||
public static final String MASTER_KILLING = "masterKilling";
|
||||
public static final String MOB_NO_KILL = "mobNoKill";
|
||||
public static final String MOB_SLOW_DETERMINISTIC = "mobSlowDeterministic";
|
||||
|
||||
public static Map<String, MonkeyFactory> FACTORIES = ImmutableMap.<String,MonkeyFactory>builder()
|
||||
.put(CALM, new CalmMonkeyFactory())
|
||||
|
@ -79,6 +81,8 @@ public abstract class MonkeyFactory {
|
|||
.put(STRESS_AM, new StressAssignmentManagerMonkeyFactory())
|
||||
.put(NO_KILL, new NoKillMonkeyFactory())
|
||||
.put(MASTER_KILLING, new MasterKillingMonkeyFactory())
|
||||
.put(MOB_NO_KILL, new MobNoKillMonkeyFactory())
|
||||
.put(MOB_SLOW_DETERMINISTIC, new MobNoKillMonkeyFactory())
|
||||
.build();
|
||||
|
||||
public static MonkeyFactory getFactory(String factoryName) {
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
@ -89,20 +90,22 @@ public class HFileLink extends FileLink {
|
|||
|
||||
private final Path archivePath;
|
||||
private final Path originPath;
|
||||
private final Path mobPath;
|
||||
private final Path tempPath;
|
||||
|
||||
/**
|
||||
* Dead simple hfile link constructor
|
||||
*/
|
||||
public HFileLink(final Path originPath, final Path tempPath,
|
||||
public HFileLink(final Path originPath, final Path tempPath, final Path mobPath,
|
||||
final Path archivePath) {
|
||||
this.tempPath = tempPath;
|
||||
this.tempPath = tempPath;
|
||||
this.originPath = originPath;
|
||||
this.mobPath = mobPath;
|
||||
this.archivePath = archivePath;
|
||||
|
||||
setLocations(originPath, tempPath, archivePath);
|
||||
setLocations(originPath, tempPath, mobPath, archivePath);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @param conf {@link Configuration} from which to extract specific archive locations
|
||||
* @param hFileLinkPattern The path ending with a HFileLink pattern. (table=region-hfile)
|
||||
|
@ -114,6 +117,8 @@ public class HFileLink extends FileLink {
|
|||
HFileArchiveUtil.getArchivePath(conf), hFileLinkPattern);
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* @param rootDir Path to the root directory where hbase files are stored
|
||||
* @param archiveDir Path to the hbase archive directory
|
||||
|
@ -125,8 +130,9 @@ public class HFileLink extends FileLink {
|
|||
Path hfilePath = getHFileLinkPatternRelativePath(hFileLinkPattern);
|
||||
Path tempPath = new Path(new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY), hfilePath);
|
||||
Path originPath = new Path(rootDir, hfilePath);
|
||||
Path mobPath = new Path(new Path(rootDir, MobConstants.MOB_DIR_NAME), hfilePath);
|
||||
Path archivePath = new Path(archiveDir, hfilePath);
|
||||
return new HFileLink(originPath, tempPath, archivePath);
|
||||
return new HFileLink(originPath, tempPath, mobPath, archivePath);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -176,6 +182,13 @@ public class HFileLink extends FileLink {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return the path of the mob hfiles.
|
||||
*/
|
||||
public Path getMobPath() {
|
||||
return this.mobPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param path Path to check.
|
||||
* @return True if the path is a HFileLink.
|
||||
*/
|
||||
|
|
|
@ -24,9 +24,13 @@ import java.io.DataInput;
|
|||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
|
@ -57,12 +61,14 @@ import org.apache.hadoop.hbase.KeyValueUtil;
|
|||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
|
||||
import org.apache.hadoop.hbase.util.BloomFilter;
|
||||
import org.apache.hadoop.hbase.util.BloomFilterUtil;
|
||||
import org.apache.hadoop.hbase.util.BloomFilterFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
@ -95,6 +101,10 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
private boolean checkRow;
|
||||
private boolean checkFamily;
|
||||
private boolean isSeekToRow = false;
|
||||
private boolean checkMobIntegrity = false;
|
||||
private Map<String, List<Path>> mobFileLocations;
|
||||
private static final int FOUND_MOB_FILES_CACHE_CAPACITY = 50;
|
||||
private static final int MISSING_MOB_FILES_CACHE_CAPACITY = 20;
|
||||
|
||||
/**
|
||||
* The row which the user wants to specify and print all the KeyValues for.
|
||||
|
@ -130,6 +140,8 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
options.addOption("w", "seekToRow", true,
|
||||
"Seek to this row and print all the kvs for this row only");
|
||||
options.addOption("s", "stats", false, "Print statistics");
|
||||
options.addOption("i", "checkMobIntegrity", false,
|
||||
"Print all cells whose mob files are missing");
|
||||
|
||||
OptionGroup files = new OptionGroup();
|
||||
files.addOption(new Option("f", "file", true,
|
||||
|
@ -158,6 +170,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
printStats = cmd.hasOption("s");
|
||||
checkRow = cmd.hasOption("k");
|
||||
checkFamily = cmd.hasOption("a");
|
||||
checkMobIntegrity = cmd.hasOption("i");
|
||||
|
||||
if (cmd.hasOption("f")) {
|
||||
files.add(new Path(cmd.getOptionValue("f")));
|
||||
|
@ -199,6 +212,12 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
files.addAll(regionFiles);
|
||||
}
|
||||
|
||||
if(checkMobIntegrity) {
|
||||
if (verbose) {
|
||||
System.out.println("checkMobIntegrity is enabled");
|
||||
}
|
||||
mobFileLocations = new HashMap<String, List<Path>>();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -255,7 +274,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
|
||||
KeyValueStatsCollector fileStats = null;
|
||||
|
||||
if (verbose || printKey || checkRow || checkFamily || printStats) {
|
||||
if (verbose || printKey || checkRow || checkFamily || printStats || checkMobIntegrity) {
|
||||
// scan over file and read key/value's and check if requested
|
||||
HFileScanner scanner = reader.getScanner(false, false, false);
|
||||
fileStats = new KeyValueStatsCollector();
|
||||
|
@ -313,6 +332,9 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
private void scanKeysValues(Path file, KeyValueStatsCollector fileStats,
|
||||
HFileScanner scanner, byte[] row) throws IOException {
|
||||
Cell pCell = null;
|
||||
FileSystem fs = FileSystem.get(getConf());
|
||||
Set<String> foundMobFiles = new LinkedHashSet<String>(FOUND_MOB_FILES_CACHE_CAPACITY);
|
||||
Set<String> missingMobFiles = new LinkedHashSet<String>(MISSING_MOB_FILES_CACHE_CAPACITY);
|
||||
do {
|
||||
Cell cell = scanner.getCell();
|
||||
if (row != null && row.length != 0) {
|
||||
|
@ -369,11 +391,87 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
+ "\n\tcurrent -> " + CellUtil.getCellKeyAsString(cell));
|
||||
}
|
||||
}
|
||||
// check if mob files are missing.
|
||||
if (checkMobIntegrity && MobUtils.isMobReferenceCell(cell)) {
|
||||
Tag tnTag = MobUtils.getTableNameTag(cell);
|
||||
if (tnTag == null) {
|
||||
System.err.println("ERROR, wrong tag format in mob reference cell "
|
||||
+ CellUtil.getCellKeyAsString(cell));
|
||||
} else if (!MobUtils.hasValidMobRefCellValue(cell)) {
|
||||
System.err.println("ERROR, wrong value format in mob reference cell "
|
||||
+ CellUtil.getCellKeyAsString(cell));
|
||||
} else {
|
||||
TableName tn = TableName.valueOf(tnTag.getValue());
|
||||
String mobFileName = MobUtils.getMobFileName(cell);
|
||||
boolean exist = mobFileExists(fs, tn, mobFileName,
|
||||
Bytes.toString(CellUtil.cloneFamily(cell)), foundMobFiles, missingMobFiles);
|
||||
if (!exist) {
|
||||
// report error
|
||||
System.err.println("ERROR, the mob file [" + mobFileName
|
||||
+ "] is missing referenced by cell " + CellUtil.getCellKeyAsString(cell));
|
||||
}
|
||||
}
|
||||
}
|
||||
pCell = cell;
|
||||
++count;
|
||||
} while (scanner.next());
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether the referenced mob file exists.
|
||||
*/
|
||||
private boolean mobFileExists(FileSystem fs, TableName tn, String mobFileName, String family,
|
||||
Set<String> foundMobFiles, Set<String> missingMobFiles) throws IOException {
|
||||
if (foundMobFiles.contains(mobFileName)) {
|
||||
return true;
|
||||
}
|
||||
if (missingMobFiles.contains(mobFileName)) {
|
||||
return false;
|
||||
}
|
||||
String tableName = tn.getNameAsString();
|
||||
List<Path> locations = mobFileLocations.get(tableName);
|
||||
if (locations == null) {
|
||||
locations = new ArrayList<Path>(2);
|
||||
locations.add(MobUtils.getMobFamilyPath(getConf(), tn, family));
|
||||
locations.add(HFileArchiveUtil.getStoreArchivePath(getConf(), tn,
|
||||
MobUtils.getMobRegionInfo(tn).getEncodedName(), family));
|
||||
mobFileLocations.put(tn.getNameAsString(), locations);
|
||||
}
|
||||
boolean exist = false;
|
||||
for (Path location : locations) {
|
||||
Path mobFilePath = new Path(location, mobFileName);
|
||||
if (fs.exists(mobFilePath)) {
|
||||
exist = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (exist) {
|
||||
evictMobFilesIfNecessary(foundMobFiles, FOUND_MOB_FILES_CACHE_CAPACITY);
|
||||
foundMobFiles.add(mobFileName);
|
||||
} else {
|
||||
evictMobFilesIfNecessary(missingMobFiles, MISSING_MOB_FILES_CACHE_CAPACITY);
|
||||
missingMobFiles.add(mobFileName);
|
||||
}
|
||||
return exist;
|
||||
}
|
||||
|
||||
/**
|
||||
* Evicts the cached mob files if the set is larger than the limit.
|
||||
*/
|
||||
private void evictMobFilesIfNecessary(Set<String> mobFileNames, int limit) {
|
||||
if (mobFileNames.size() < limit) {
|
||||
return;
|
||||
}
|
||||
int index = 0;
|
||||
int evict = limit / 2;
|
||||
Iterator<String> fileNamesItr = mobFileNames.iterator();
|
||||
while (index < evict && fileNamesItr.hasNext()) {
|
||||
fileNamesItr.next();
|
||||
fileNamesItr.remove();
|
||||
index++;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Format a string of the form "k1=v1, k2=v2, ..." into separate lines
|
||||
* with a four-space indentation.
|
||||
|
|
|
@ -0,0 +1,107 @@
|
|||
/**
|
||||
* 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.master;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ScheduledChore;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.LockTimeoutException;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
|
||||
/**
|
||||
* The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
|
||||
* mob files.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ExpiredMobFileCleanerChore extends ScheduledChore {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ExpiredMobFileCleanerChore.class);
|
||||
private final HMaster master;
|
||||
private TableLockManager tableLockManager;
|
||||
private ExpiredMobFileCleaner cleaner;
|
||||
|
||||
public ExpiredMobFileCleanerChore(HMaster master) {
|
||||
super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, master.getConfiguration()
|
||||
.getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), master
|
||||
.getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD,
|
||||
MobConstants.DEFAULT_MOB_CLEANER_PERIOD), TimeUnit.SECONDS);
|
||||
this.master = master;
|
||||
this.tableLockManager = master.getTableLockManager();
|
||||
cleaner = new ExpiredMobFileCleaner();
|
||||
cleaner.setConf(master.getConfiguration());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void chore() {
|
||||
try {
|
||||
TableDescriptors htds = master.getTableDescriptors();
|
||||
Map<String, HTableDescriptor> map = htds.getAll();
|
||||
for (HTableDescriptor htd : map.values()) {
|
||||
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
|
||||
if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) {
|
||||
// clean only for mob-enabled column.
|
||||
// obtain a read table lock before cleaning, synchronize with MobFileCompactionChore.
|
||||
boolean tableLocked = false;
|
||||
TableLock lock = null;
|
||||
try {
|
||||
// the tableLockManager might be null in testing. In that case, it is lock-free.
|
||||
if (tableLockManager != null) {
|
||||
lock = tableLockManager.readLock(MobUtils.getTableLockName(htd.getTableName()),
|
||||
"Run ExpiredMobFileCleanerChore");
|
||||
lock.acquire();
|
||||
}
|
||||
tableLocked = true;
|
||||
cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd);
|
||||
} catch (LockTimeoutException e) {
|
||||
LOG.info("Fail to acquire the lock because of timeout, maybe a"
|
||||
+ " MobFileCompactor is running", e);
|
||||
} catch (Exception e) {
|
||||
LOG.error(
|
||||
"Fail to clean the expired mob files for the column " + hcd.getNameAsString()
|
||||
+ " in the table " + htd.getNameAsString(), e);
|
||||
} finally {
|
||||
if (lock != null && tableLocked) {
|
||||
try {
|
||||
lock.release();
|
||||
} catch (IOException e) {
|
||||
LOG.error(
|
||||
"Fail to release the read lock for the table " + htd.getNameAsString(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Fail to clean the expired mob files", e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -36,6 +36,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
@ -105,6 +106,7 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
|
|||
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
|
||||
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
|
@ -112,6 +114,7 @@ import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
|
|||
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
|
||||
|
@ -129,6 +132,7 @@ import org.apache.hadoop.hbase.util.EncryptionTest;
|
|||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
import org.apache.hadoop.hbase.util.HasThread;
|
||||
import org.apache.hadoop.hbase.util.IdLock;
|
||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -281,6 +285,14 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
CatalogJanitor catalogJanitorChore;
|
||||
private LogCleaner logCleaner;
|
||||
private HFileCleaner hfileCleaner;
|
||||
private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
|
||||
private MobCompactionChore mobCompactChore;
|
||||
private MasterMobCompactionThread mobCompactThread;
|
||||
// used to synchronize the mobCompactionStates
|
||||
private final IdLock mobCompactionLock = new IdLock();
|
||||
// save the information of mob compactions in tables.
|
||||
// the key is table name, the value is the number of compactions in that table.
|
||||
private Map<TableName, AtomicInteger> mobCompactionStates = Maps.newConcurrentMap();
|
||||
|
||||
MasterCoprocessorHost cpHost;
|
||||
|
||||
|
@ -794,6 +806,21 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
zooKeeper.checkAndSetZNodeAcls();
|
||||
|
||||
status.setStatus("Calling postStartMaster coprocessors");
|
||||
|
||||
this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this);
|
||||
getChoreService().scheduleChore(expiredMobFileCleanerChore);
|
||||
|
||||
int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
|
||||
MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD);
|
||||
if (mobCompactionPeriod > 0) {
|
||||
this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod);
|
||||
getChoreService().scheduleChore(mobCompactChore);
|
||||
} else {
|
||||
LOG
|
||||
.info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled");
|
||||
}
|
||||
this.mobCompactThread = new MasterMobCompactionThread(this);
|
||||
|
||||
if (this.cpHost != null) {
|
||||
// don't let cp initialization errors kill the master
|
||||
try {
|
||||
|
@ -1127,6 +1154,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
}
|
||||
|
||||
private void stopChores() {
|
||||
if (this.expiredMobFileCleanerChore != null) {
|
||||
this.expiredMobFileCleanerChore.cancel(true);
|
||||
}
|
||||
if (this.mobCompactChore != null) {
|
||||
this.mobCompactChore.cancel(true);
|
||||
}
|
||||
if (this.balancerChore != null) {
|
||||
this.balancerChore.cancel(true);
|
||||
}
|
||||
|
@ -1142,6 +1175,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
if (this.clusterStatusPublisherChore != null){
|
||||
clusterStatusPublisherChore.cancel(true);
|
||||
}
|
||||
if (this.mobCompactThread != null) {
|
||||
this.mobCompactThread.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2538,6 +2574,71 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
return getClusterStatus().getLastMajorCompactionTsForRegion(regionName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the mob file compaction state for a specific table.
|
||||
* Whether all the mob files are selected is known during the compaction execution, but
|
||||
* the statistic is done just before compaction starts, it is hard to know the compaction
|
||||
* type at that time, so the rough statistics are chosen for the mob file compaction. Only two
|
||||
* compaction states are available, CompactionState.MAJOR_AND_MINOR and CompactionState.NONE.
|
||||
* @param tableName The current table name.
|
||||
* @return If a given table is in mob file compaction now.
|
||||
*/
|
||||
public CompactionState getMobCompactionState(TableName tableName) {
|
||||
AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
|
||||
if (compactionsCount != null && compactionsCount.get() != 0) {
|
||||
return CompactionState.MAJOR_AND_MINOR;
|
||||
}
|
||||
return CompactionState.NONE;
|
||||
}
|
||||
|
||||
public void reportMobCompactionStart(TableName tableName) throws IOException {
|
||||
IdLock.Entry lockEntry = null;
|
||||
try {
|
||||
lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
|
||||
AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
|
||||
if (compactionsCount == null) {
|
||||
compactionsCount = new AtomicInteger(0);
|
||||
mobCompactionStates.put(tableName, compactionsCount);
|
||||
}
|
||||
compactionsCount.incrementAndGet();
|
||||
} finally {
|
||||
if (lockEntry != null) {
|
||||
mobCompactionLock.releaseLockEntry(lockEntry);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void reportMobCompactionEnd(TableName tableName) throws IOException {
|
||||
IdLock.Entry lockEntry = null;
|
||||
try {
|
||||
lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
|
||||
AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
|
||||
if (compactionsCount != null) {
|
||||
int count = compactionsCount.decrementAndGet();
|
||||
// remove the entry if the count is 0.
|
||||
if (count == 0) {
|
||||
mobCompactionStates.remove(tableName);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (lockEntry != null) {
|
||||
mobCompactionLock.releaseLockEntry(lockEntry);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Requests mob compaction.
|
||||
* @param tableName The table the compact.
|
||||
* @param columns The compacted columns.
|
||||
* @param allFiles Whether add all mob files into the compaction.
|
||||
*/
|
||||
public void requestMobCompaction(TableName tableName,
|
||||
List<HColumnDescriptor> columns, boolean allFiles) throws IOException {
|
||||
mobCompactThread.requestMobCompaction(conf, fs, tableName, columns,
|
||||
tableLockManager, allFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized,
|
||||
* false is returned.
|
||||
|
|
|
@ -49,6 +49,8 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
|
@ -598,7 +600,7 @@ public class MasterFileSystem {
|
|||
// @see HRegion.checkRegioninfoOnFilesystem()
|
||||
}
|
||||
|
||||
public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName)
|
||||
public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName, boolean hasMob)
|
||||
throws IOException {
|
||||
// archive family store files
|
||||
Path tableDir = FSUtils.getTableDir(rootdir, region.getTable());
|
||||
|
@ -615,6 +617,24 @@ public class MasterFileSystem {
|
|||
+ ")");
|
||||
}
|
||||
}
|
||||
|
||||
// archive and delete mob files
|
||||
if (hasMob) {
|
||||
Path mobTableDir =
|
||||
FSUtils.getTableDir(new Path(getRootDir(), MobConstants.MOB_DIR_NAME), region.getTable());
|
||||
HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(region.getTable());
|
||||
Path mobFamilyDir =
|
||||
new Path(mobTableDir,
|
||||
new Path(mobRegionInfo.getEncodedName(), Bytes.toString(familyName)));
|
||||
// archive mob family store files
|
||||
MobUtils.archiveMobStoreFiles(conf, fs, mobRegionInfo, mobFamilyDir, familyName);
|
||||
|
||||
if (!fs.delete(mobFamilyDir, true)) {
|
||||
throw new IOException("Could not delete mob store files for family "
|
||||
+ Bytes.toString(familyName) + " from FileSystem region "
|
||||
+ mobRegionInfo.getRegionNameAsString() + "(" + mobRegionInfo.getEncodedName() + ")");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
|
|
|
@ -0,0 +1,184 @@
|
|||
/**
|
||||
*
|
||||
* 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.master;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.SynchronousQueue;
|
||||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
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.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
/**
|
||||
* The mob compaction thread used in {@link MasterRpcServices}
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MasterMobCompactionThread {
|
||||
static final Log LOG = LogFactory.getLog(MasterMobCompactionThread.class);
|
||||
private final HMaster master;
|
||||
private final Configuration conf;
|
||||
private final ExecutorService mobCompactorPool;
|
||||
private final ExecutorService masterMobPool;
|
||||
|
||||
public MasterMobCompactionThread(HMaster master) {
|
||||
this.master = master;
|
||||
this.conf = master.getConfiguration();
|
||||
final String n = Thread.currentThread().getName();
|
||||
// this pool is used to run the mob compaction
|
||||
this.masterMobPool = new ThreadPoolExecutor(1, 2, 60, TimeUnit.SECONDS,
|
||||
new SynchronousQueue<Runnable>(), new ThreadFactory() {
|
||||
@Override
|
||||
public Thread newThread(Runnable r) {
|
||||
Thread t = new Thread(r);
|
||||
t.setName(n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime());
|
||||
return t;
|
||||
}
|
||||
});
|
||||
((ThreadPoolExecutor) this.masterMobPool).allowCoreThreadTimeOut(true);
|
||||
// this pool is used in the mob compaction to compact the mob files by partitions
|
||||
// in parallel
|
||||
this.mobCompactorPool = MobUtils
|
||||
.createMobCompactorThreadPool(master.getConfiguration());
|
||||
}
|
||||
|
||||
/**
|
||||
* Requests mob compaction
|
||||
* @param conf The Configuration
|
||||
* @param fs The file system
|
||||
* @param tableName The table the compact
|
||||
* @param columns The column descriptors
|
||||
* @param tableLockManager The tableLock manager
|
||||
* @param allFiles Whether add all mob files into the compaction.
|
||||
*/
|
||||
public void requestMobCompaction(Configuration conf, FileSystem fs, TableName tableName,
|
||||
List<HColumnDescriptor> columns, TableLockManager tableLockManager, boolean allFiles)
|
||||
throws IOException {
|
||||
master.reportMobCompactionStart(tableName);
|
||||
try {
|
||||
masterMobPool.execute(new CompactionRunner(fs, tableName, columns, tableLockManager,
|
||||
allFiles, mobCompactorPool));
|
||||
} catch (RejectedExecutionException e) {
|
||||
// in case the request is rejected by the pool
|
||||
try {
|
||||
master.reportMobCompactionEnd(tableName);
|
||||
} catch (IOException e1) {
|
||||
LOG.error("Failed to mark end of mob compation", e1);
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("The mob compaction is requested for the columns " + columns
|
||||
+ " of the table " + tableName.getNameAsString());
|
||||
}
|
||||
}
|
||||
|
||||
private class CompactionRunner implements Runnable {
|
||||
private FileSystem fs;
|
||||
private TableName tableName;
|
||||
private List<HColumnDescriptor> hcds;
|
||||
private TableLockManager tableLockManager;
|
||||
private boolean allFiles;
|
||||
private ExecutorService pool;
|
||||
|
||||
public CompactionRunner(FileSystem fs, TableName tableName, List<HColumnDescriptor> hcds,
|
||||
TableLockManager tableLockManager, boolean allFiles, ExecutorService pool) {
|
||||
super();
|
||||
this.fs = fs;
|
||||
this.tableName = tableName;
|
||||
this.hcds = hcds;
|
||||
this.tableLockManager = tableLockManager;
|
||||
this.allFiles = allFiles;
|
||||
this.pool = pool;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
MobUtils.doMobCompaction(conf, fs, tableName, hcd, pool, tableLockManager,
|
||||
allFiles);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to perform the mob compaction", e);
|
||||
} finally {
|
||||
try {
|
||||
master.reportMobCompactionEnd(tableName);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to mark end of mob compation", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Only interrupt once it's done with a run through the work loop.
|
||||
*/
|
||||
private void interruptIfNecessary() {
|
||||
mobCompactorPool.shutdown();
|
||||
masterMobPool.shutdown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait for all the threads finish.
|
||||
*/
|
||||
private void join() {
|
||||
waitFor(mobCompactorPool, "Mob Compaction Thread");
|
||||
waitFor(masterMobPool, "Region Server Mob Compaction Thread");
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the MasterMobCompactionThread.
|
||||
*/
|
||||
public void close() {
|
||||
interruptIfNecessary();
|
||||
join();
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait for thread finish.
|
||||
* @param t the thread to wait
|
||||
* @param name the thread name.
|
||||
*/
|
||||
private void waitFor(ExecutorService t, String name) {
|
||||
boolean done = false;
|
||||
while (!done) {
|
||||
try {
|
||||
done = t.awaitTermination(60, TimeUnit.SECONDS);
|
||||
LOG.info("Waiting for " + name + " to finish...");
|
||||
if (!done) {
|
||||
t.shutdownNow();
|
||||
}
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("Interrupted waiting for " + name + " to finish...");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
|
|||
import org.apache.hadoop.hbase.ipc.QosPriority;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
|
||||
|
@ -51,6 +53,10 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.*;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
|
||||
|
@ -1394,6 +1400,104 @@ public class MasterRpcServices extends RSRpcServices
|
|||
return response.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Compact a region on the master.
|
||||
*
|
||||
* @param controller the RPC controller
|
||||
* @param request the request
|
||||
* @throws ServiceException
|
||||
*/
|
||||
@Override
|
||||
@QosPriority(priority=HConstants.ADMIN_QOS)
|
||||
public CompactRegionResponse compactRegion(final RpcController controller,
|
||||
final CompactRegionRequest request) throws ServiceException {
|
||||
try {
|
||||
master.checkInitialized();
|
||||
byte[] regionName = request.getRegion().getValue().toByteArray();
|
||||
TableName tableName = HRegionInfo.getTable(regionName);
|
||||
// if the region is a mob region, do the mob file compaction.
|
||||
if (MobUtils.isMobRegionName(tableName, regionName)) {
|
||||
return compactMob(request, tableName);
|
||||
} else {
|
||||
return super.compactRegion(controller, request);
|
||||
}
|
||||
} catch (IOException ie) {
|
||||
throw new ServiceException(ie);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@QosPriority(priority=HConstants.ADMIN_QOS)
|
||||
public GetRegionInfoResponse getRegionInfo(final RpcController controller,
|
||||
final GetRegionInfoRequest request) throws ServiceException {
|
||||
byte[] regionName = request.getRegion().getValue().toByteArray();
|
||||
TableName tableName = HRegionInfo.getTable(regionName);
|
||||
if (MobUtils.isMobRegionName(tableName, regionName)) {
|
||||
// a dummy region info contains the compaction state.
|
||||
HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
|
||||
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
|
||||
builder.setRegionInfo(HRegionInfo.convert(mobRegionInfo));
|
||||
if (request.hasCompactionState() && request.getCompactionState()) {
|
||||
builder.setCompactionState(master.getMobCompactionState(tableName));
|
||||
}
|
||||
return builder.build();
|
||||
} else {
|
||||
return super.getRegionInfo(controller, request);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts the mob files in the current table.
|
||||
* @param request the request.
|
||||
* @param tableName the current table name.
|
||||
* @return The response of the mob file compaction.
|
||||
* @throws IOException
|
||||
*/
|
||||
private CompactRegionResponse compactMob(final CompactRegionRequest request,
|
||||
TableName tableName) throws IOException {
|
||||
if (!master.getTableStateManager().isTableState(tableName, TableState.State.ENABLED)) {
|
||||
throw new DoNotRetryIOException("Table " + tableName + " is not enabled");
|
||||
}
|
||||
boolean allFiles = false;
|
||||
List<HColumnDescriptor> compactedColumns = new ArrayList<HColumnDescriptor>();
|
||||
HColumnDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies();
|
||||
byte[] family = null;
|
||||
if (request.hasFamily()) {
|
||||
family = request.getFamily().toByteArray();
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
if (Bytes.equals(family, hcd.getName())) {
|
||||
if (!hcd.isMobEnabled()) {
|
||||
LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family");
|
||||
throw new DoNotRetryIOException("Column family " + hcd.getNameAsString()
|
||||
+ " is not a mob column family");
|
||||
}
|
||||
compactedColumns.add(hcd);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
if (hcd.isMobEnabled()) {
|
||||
compactedColumns.add(hcd);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (compactedColumns.isEmpty()) {
|
||||
LOG.error("No mob column families are assigned in the mob compaction");
|
||||
throw new DoNotRetryIOException(
|
||||
"No mob column families are assigned in the mob compaction");
|
||||
}
|
||||
if (request.hasMajor() && request.getMajor()) {
|
||||
allFiles = true;
|
||||
}
|
||||
String familyLogMsg = (family != null) ? Bytes.toString(family) : "";
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("User-triggered mob compaction requested for table: "
|
||||
+ tableName.getNameAsString() + " for column family: " + familyLogMsg);
|
||||
}
|
||||
master.requestMobCompaction(tableName, compactedColumns, allFiles);
|
||||
return CompactRegionResponse.newBuilder().build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller,
|
||||
IsBalancerEnabledRequest request) throws ServiceException {
|
||||
|
|
|
@ -0,0 +1,93 @@
|
|||
/**
|
||||
*
|
||||
* 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.master;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ScheduledChore;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
|
||||
/**
|
||||
* The Class MobCompactChore for running compaction regularly to merge small mob files.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobCompactionChore extends ScheduledChore {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(MobCompactionChore.class);
|
||||
private HMaster master;
|
||||
private TableLockManager tableLockManager;
|
||||
private ExecutorService pool;
|
||||
|
||||
public MobCompactionChore(HMaster master, int period) {
|
||||
// use the period as initial delay.
|
||||
super(master.getServerName() + "-MobCompactionChore", master, period, period, TimeUnit.SECONDS);
|
||||
this.master = master;
|
||||
this.tableLockManager = master.getTableLockManager();
|
||||
this.pool = MobUtils.createMobCompactorThreadPool(master.getConfiguration());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void chore() {
|
||||
try {
|
||||
TableDescriptors htds = master.getTableDescriptors();
|
||||
Map<String, HTableDescriptor> map = htds.getAll();
|
||||
for (HTableDescriptor htd : map.values()) {
|
||||
if (!master.getTableStateManager().isTableState(htd.getTableName(),
|
||||
TableState.State.ENABLED)) {
|
||||
continue;
|
||||
}
|
||||
boolean reported = false;
|
||||
try {
|
||||
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
|
||||
if (!hcd.isMobEnabled()) {
|
||||
continue;
|
||||
}
|
||||
if (!reported) {
|
||||
master.reportMobCompactionStart(htd.getTableName());
|
||||
reported = true;
|
||||
}
|
||||
MobUtils.doMobCompaction(master.getConfiguration(), master.getFileSystem(),
|
||||
htd.getTableName(), hcd, pool, tableLockManager, false);
|
||||
}
|
||||
} finally {
|
||||
if (reported) {
|
||||
master.reportMobCompactionEnd(htd.getTableName());
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to compact mob files", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void cleanup() {
|
||||
super.cleanup();
|
||||
pool.shutdown();
|
||||
}
|
||||
}
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
/**
|
||||
|
@ -64,6 +65,11 @@ public class HFileLinkCleaner extends BaseHFileCleanerDelegate {
|
|||
if (fs.exists(hfilePath)) {
|
||||
return false;
|
||||
}
|
||||
// check whether the HFileLink still exists in mob dir.
|
||||
hfilePath = HFileLink.getHFileFromBackReference(MobUtils.getMobHome(getConf()), filePath);
|
||||
if (fs.exists(hfilePath)) {
|
||||
return false;
|
||||
}
|
||||
hfilePath = HFileLink.getHFileFromBackReference(FSUtils.getRootDir(getConf()), filePath);
|
||||
return !fs.exists(hfilePath);
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -313,7 +313,7 @@ public class AddColumnFamilyProcedure
|
|||
// Remove the column family from file system and update the table descriptor to
|
||||
// the before-add-column-family-state
|
||||
MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(env, tableName,
|
||||
getRegionInfoList(env), cfDescriptor.getName());
|
||||
getRegionInfoList(env), cfDescriptor.getName(), cfDescriptor.isMobEnabled());
|
||||
|
||||
env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
|
||||
|
||||
|
@ -405,4 +405,4 @@ public class AddColumnFamilyProcedure
|
|||
}
|
||||
return regionInfoList;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,6 +57,7 @@ public class DeleteColumnFamilyProcedure
|
|||
private HTableDescriptor unmodifiedHTableDescriptor;
|
||||
private TableName tableName;
|
||||
private byte [] familyName;
|
||||
private boolean hasMob;
|
||||
private UserGroupInformation user;
|
||||
|
||||
private List<HRegionInfo> regionInfoList;
|
||||
|
@ -289,6 +290,9 @@ public class DeleteColumnFamilyProcedure
|
|||
throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
|
||||
+ "' is the only column family in the table, so it cannot be deleted");
|
||||
}
|
||||
|
||||
// whether mob family
|
||||
hasMob = unmodifiedHTableDescriptor.getFamily(familyName).isMobEnabled();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -340,7 +344,7 @@ public class DeleteColumnFamilyProcedure
|
|||
**/
|
||||
private void deleteFromFs(final MasterProcedureEnv env) throws IOException {
|
||||
MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(env, tableName,
|
||||
getRegionInfoList(env), familyName);
|
||||
getRegionInfoList(env), familyName, hasMob);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,8 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.master.procedure;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
|
@ -27,16 +27,17 @@ import java.util.List;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
|
@ -44,15 +45,17 @@ import org.apache.hadoop.hbase.client.ResultScanner;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.exceptions.HBaseException;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
||||
|
@ -342,10 +345,32 @@ public class DeleteTableProcedure
|
|||
LOG.debug("Table '" + tableName + "' archived!");
|
||||
}
|
||||
|
||||
// Archive the mob data if there is a mob-enabled column
|
||||
HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||
boolean hasMob = MobUtils.hasMobColumns(htd);
|
||||
Path mobTableDir = null;
|
||||
if (hasMob) {
|
||||
// Archive mob data
|
||||
mobTableDir = FSUtils.getTableDir(new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME),
|
||||
tableName);
|
||||
Path regionDir =
|
||||
new Path(mobTableDir, MobUtils.getMobRegionInfo(tableName).getEncodedName());
|
||||
if (fs.exists(regionDir)) {
|
||||
HFileArchiver.archiveRegion(fs, mfs.getRootDir(), mobTableDir, regionDir);
|
||||
}
|
||||
}
|
||||
|
||||
// Delete table directory from FS (temp directory)
|
||||
if (!fs.delete(tempTableDir, true) && fs.exists(tempTableDir)) {
|
||||
throw new IOException("Couldn't delete " + tempTableDir);
|
||||
}
|
||||
|
||||
// Delete the table directory where the mob files are saved
|
||||
if (hasMob && mobTableDir != null && fs.exists(mobTableDir)) {
|
||||
if (!fs.delete(mobTableDir, true)) {
|
||||
throw new IOException("Couldn't delete mob dir " + mobTableDir);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -407,4 +432,4 @@ public class DeleteTableProcedure
|
|||
ProcedureSyncWait.getMasterQuotaManager(env).removeTableFromNamespaceQuota(tableName);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -91,7 +91,8 @@ public final class MasterDDLOperationHelper {
|
|||
final MasterProcedureEnv env,
|
||||
final TableName tableName,
|
||||
List<HRegionInfo> regionInfoList,
|
||||
final byte[] familyName) throws IOException {
|
||||
final byte[] familyName,
|
||||
boolean hasMob) throws IOException {
|
||||
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Removing family=" + Bytes.toString(familyName) + " from table=" + tableName);
|
||||
|
@ -101,7 +102,7 @@ public final class MasterDDLOperationHelper {
|
|||
}
|
||||
for (HRegionInfo hri : regionInfoList) {
|
||||
// Delete the family directory in FS for all the regions one by one
|
||||
mfs.deleteFamilyFromFS(hri, familyName);
|
||||
mfs.deleteFamilyFromFS(hri, familyName, hasMob);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -369,7 +369,8 @@ public class ModifyTableProcedure
|
|||
env,
|
||||
getTableName(),
|
||||
getRegionInfoList(env),
|
||||
familyName);
|
||||
familyName,
|
||||
oldHTableDescriptor.getFamily(familyName).isMobEnabled());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
@ -35,10 +36,12 @@ import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
|||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionListener;
|
||||
import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -90,6 +93,11 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
|
|||
if (RegionReplicaUtil.isDefaultReplica(hri)) {
|
||||
regions.add(hri);
|
||||
}
|
||||
// if it's the first region, add the mob region
|
||||
if (Bytes.equals(hri.getStartKey(), HConstants.EMPTY_START_ROW)) {
|
||||
HRegionInfo mobRegion = MobUtils.getMobRegionInfo(hri.getTable());
|
||||
regions.add(mobRegion);
|
||||
}
|
||||
}
|
||||
|
||||
// 2. for each region, write all the info to disk
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
|
||||
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
||||
|
@ -168,10 +169,17 @@ public final class MasterSnapshotVerifier {
|
|||
}
|
||||
|
||||
String errorMsg = "";
|
||||
if (regionManifests.size() != regions.size()) {
|
||||
boolean hasMobStore = false;
|
||||
// the mob region is a dummy region, it's not a real region in HBase.
|
||||
// the mob region has a special name, it could be found by the region name.
|
||||
if (regionManifests.get(MobUtils.getMobRegionInfo(tableName).getEncodedName()) != null) {
|
||||
hasMobStore = true;
|
||||
}
|
||||
int realRegionCount = hasMobStore ? regionManifests.size() - 1 : regionManifests.size();
|
||||
if (realRegionCount != regions.size()) {
|
||||
errorMsg = "Regions moved during the snapshot '" +
|
||||
ClientSnapshotDescriptionUtils.toString(snapshot) + "'. expected=" +
|
||||
regions.size() + " snapshotted=" + regionManifests.size() + ".";
|
||||
regions.size() + " snapshotted=" + realRegionCount + ".";
|
||||
LOG.error(errorMsg);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,114 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
/**
|
||||
* Cached mob file.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class CachedMobFile extends MobFile implements Comparable<CachedMobFile> {
|
||||
|
||||
private long accessCount;
|
||||
private AtomicLong referenceCount = new AtomicLong(0);
|
||||
|
||||
public CachedMobFile(StoreFile sf) {
|
||||
super(sf);
|
||||
}
|
||||
|
||||
public static CachedMobFile create(FileSystem fs, Path path, Configuration conf,
|
||||
CacheConfig cacheConf) throws IOException {
|
||||
StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE);
|
||||
return new CachedMobFile(sf);
|
||||
}
|
||||
|
||||
public void access(long accessCount) {
|
||||
this.accessCount = accessCount;
|
||||
}
|
||||
|
||||
public int compareTo(CachedMobFile that) {
|
||||
if (this.accessCount == that.accessCount) return 0;
|
||||
return this.accessCount < that.accessCount ? 1 : -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (!(obj instanceof CachedMobFile)) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((CachedMobFile) obj) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return (int)(accessCount ^ (accessCount >>> 32));
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens the mob file if it's not opened yet and increases the reference.
|
||||
* It's not thread-safe. Use MobFileCache.openFile() instead.
|
||||
* The reader of the mob file is just opened when it's not opened no matter how many times
|
||||
* this open() method is invoked.
|
||||
* The reference is a counter that how many times this reader is referenced. When the
|
||||
* reference is 0, this reader is closed.
|
||||
*/
|
||||
@Override
|
||||
public void open() throws IOException {
|
||||
super.open();
|
||||
referenceCount.incrementAndGet();
|
||||
}
|
||||
|
||||
/**
|
||||
* Decreases the reference of the underlying reader for the mob file.
|
||||
* It's not thread-safe. Use MobFileCache.closeFile() instead.
|
||||
* This underlying reader isn't closed until the reference is 0.
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
long refs = referenceCount.decrementAndGet();
|
||||
if (refs == 0) {
|
||||
super.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the reference of the current mob file.
|
||||
* Internal usage, currently it's for testing.
|
||||
* @return The reference of the current mob file.
|
||||
*/
|
||||
public long getReferenceCount() {
|
||||
return this.referenceCount.longValue();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,300 @@
|
|||
/**
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
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.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.HMobStore;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.MobCompactionStoreScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* Compact passed set of files in the mob-enabled column family.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(DefaultMobStoreCompactor.class);
|
||||
private long mobSizeThreshold;
|
||||
private HMobStore mobStore;
|
||||
public DefaultMobStoreCompactor(Configuration conf, Store store) {
|
||||
super(conf, store);
|
||||
// The mob cells reside in the mob-enabled column family which is held by HMobStore.
|
||||
// During the compaction, the compactor reads the cells from the mob files and
|
||||
// probably creates new mob files. All of these operations are included in HMobStore,
|
||||
// so we need to cast the Store to HMobStore.
|
||||
if (!(store instanceof HMobStore)) {
|
||||
throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
|
||||
}
|
||||
mobStore = (HMobStore) store;
|
||||
mobSizeThreshold = store.getFamily().getMobThreshold();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a writer for a new file in a temporary directory.
|
||||
* @param fd The file details.
|
||||
* @param smallestReadPoint The smallest mvcc readPoint across all the scanners in this region.
|
||||
* @return Writer for a new StoreFile in the tmp dir.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
protected Writer createTmpWriter(FileDetails fd, long smallestReadPoint) throws IOException {
|
||||
// make this writer with tags always because of possible new cells with tags.
|
||||
StoreFile.Writer writer = store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression,
|
||||
true, true, true);
|
||||
return writer;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
|
||||
ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(store.getFamily().getMaxVersions());
|
||||
if (scanType == ScanType.COMPACT_DROP_DELETES) {
|
||||
// In major compaction, we need to write the delete markers to del files, so we have to
|
||||
// retain the them in scanning.
|
||||
scanType = ScanType.COMPACT_RETAIN_DELETES;
|
||||
return new MobCompactionStoreScanner(store, store.getScanInfo(), scan, scanners,
|
||||
scanType, smallestReadPoint, earliestPutTs, true);
|
||||
} else {
|
||||
return new MobCompactionStoreScanner(store, store.getScanInfo(), scan, scanners,
|
||||
scanType, smallestReadPoint, earliestPutTs, false);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO refactor to take advantage of the throughput controller.
|
||||
|
||||
/**
|
||||
* Performs compaction on a column family with the mob flag enabled.
|
||||
* This is for when the mob threshold size has changed or if the mob
|
||||
* column family mode has been toggled via an alter table statement.
|
||||
* Compacts the files by the following rules.
|
||||
* 1. If the cell has a mob reference tag, the cell's value is the path of the mob file.
|
||||
* <ol>
|
||||
* <li>
|
||||
* If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
|
||||
* directly copy the (with mob tag) cell into the new store file.
|
||||
* </li>
|
||||
* <li>
|
||||
* Otherwise, retrieve the mob cell from the mob file, and writes a copy of the cell into
|
||||
* the new store file.
|
||||
* </li>
|
||||
* </ol>
|
||||
* 2. If the cell doesn't have a reference tag.
|
||||
* <ol>
|
||||
* <li>
|
||||
* If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
|
||||
* write this cell to a mob file, and write the path of this mob file to the store file.
|
||||
* </li>
|
||||
* <li>
|
||||
* Otherwise, directly write this cell into the store file.
|
||||
* </li>
|
||||
* </ol>
|
||||
* In the mob compaction, the {@link MobCompactionStoreScanner} is used as a scanner
|
||||
* which could output the normal cells and delete markers together when required.
|
||||
* After the major compaction on the normal hfiles, we have a guarantee that we have purged all
|
||||
* deleted or old version mob refs, and the delete markers are written to a del file with the
|
||||
* suffix _del. Because of this, it is safe to use the del file in the mob compaction.
|
||||
* The mob compaction doesn't take place in the normal hfiles, it occurs directly in the
|
||||
* mob files. When the small mob files are merged into bigger ones, the del file is added into
|
||||
* the scanner to filter the deleted cells.
|
||||
* @param fd File details
|
||||
* @param scanner Where to read from.
|
||||
* @param writer Where to write to.
|
||||
* @param smallestReadPoint Smallest read point.
|
||||
* @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= smallestReadPoint
|
||||
* @param throughputController The compaction throughput controller.
|
||||
* @param major Is a major compaction.
|
||||
* @return Whether compaction ended; false if it was interrupted for any reason.
|
||||
*/
|
||||
@Override
|
||||
protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
|
||||
long smallestReadPoint, boolean cleanSeqId,
|
||||
CompactionThroughputController throughputController, boolean major) throws IOException {
|
||||
if (!(scanner instanceof MobCompactionStoreScanner)) {
|
||||
throw new IllegalArgumentException(
|
||||
"The scanner should be an instance of MobCompactionStoreScanner");
|
||||
}
|
||||
MobCompactionStoreScanner compactionScanner = (MobCompactionStoreScanner) scanner;
|
||||
int bytesWritten = 0;
|
||||
// Since scanner.next() can return 'false' but still be delivering data,
|
||||
// we have to use a do/while loop.
|
||||
List<Cell> cells = new ArrayList<Cell>();
|
||||
// Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
|
||||
int closeCheckInterval = HStore.getCloseCheckInterval();
|
||||
boolean hasMore;
|
||||
Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
|
||||
byte[] fileName = null;
|
||||
StoreFile.Writer mobFileWriter = null, delFileWriter = null;
|
||||
long mobCells = 0, deleteMarkersCount = 0;
|
||||
Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, store.getTableName().getName());
|
||||
long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0;
|
||||
long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0;
|
||||
try {
|
||||
try {
|
||||
// If the mob file writer could not be created, directly write the cell to the store file.
|
||||
mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
|
||||
store.getFamily().getCompression(), store.getRegionInfo().getStartKey());
|
||||
fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to create mob writer, "
|
||||
+ "we will continue the compaction by writing MOB cells directly in store files", e);
|
||||
}
|
||||
delFileWriter = mobStore.createDelFileWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
|
||||
store.getFamily().getCompression(), store.getRegionInfo().getStartKey());
|
||||
ScannerContext scannerContext =
|
||||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
||||
do {
|
||||
hasMore = compactionScanner.next(cells, scannerContext);
|
||||
for (Cell c : cells) {
|
||||
if (compactionScanner.isOutputDeleteMarkers() && CellUtil.isDelete(c)) {
|
||||
delFileWriter.append(c);
|
||||
deleteMarkersCount++;
|
||||
} else if (mobFileWriter == null || c.getTypeByte() != KeyValue.Type.Put.getCode()) {
|
||||
// If the mob file writer is null or the kv type is not put, directly write the cell
|
||||
// to the store file.
|
||||
writer.append(c);
|
||||
} else if (MobUtils.isMobReferenceCell(c)) {
|
||||
if (MobUtils.hasValidMobRefCellValue(c)) {
|
||||
int size = MobUtils.getMobValueLength(c);
|
||||
if (size > mobSizeThreshold) {
|
||||
// If the value size is larger than the threshold, it's regarded as a mob. Since
|
||||
// its value is already in the mob file, directly write this cell to the store file
|
||||
writer.append(c);
|
||||
} else {
|
||||
// If the value is not larger than the threshold, it's not regarded a mob. Retrieve
|
||||
// the mob cell from the mob file, and write it back to the store file.
|
||||
Cell mobCell = mobStore.resolve(c, false);
|
||||
if (mobCell.getValueLength() != 0) {
|
||||
// put the mob data back to the store file
|
||||
CellUtil.setSequenceId(mobCell, c.getSequenceId());
|
||||
writer.append(mobCell);
|
||||
cellsCountCompactedFromMob++;
|
||||
cellsSizeCompactedFromMob += mobCell.getValueLength();
|
||||
} else {
|
||||
// If the value of a file is empty, there might be issues when retrieving,
|
||||
// directly write the cell to the store file, and leave it to be handled by the
|
||||
// next compaction.
|
||||
writer.append(c);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
LOG.warn("The value format of the KeyValue " + c
|
||||
+ " is wrong, its length is less than " + Bytes.SIZEOF_INT);
|
||||
writer.append(c);
|
||||
}
|
||||
} else if (c.getValueLength() <= mobSizeThreshold) {
|
||||
//If value size of a cell is not larger than the threshold, directly write to store file
|
||||
writer.append(c);
|
||||
} else {
|
||||
// If the value size of a cell is larger than the threshold, it's regarded as a mob,
|
||||
// write this cell to a mob file, and write the path to the store file.
|
||||
mobCells++;
|
||||
// append the original keyValue in the mob file.
|
||||
mobFileWriter.append(c);
|
||||
KeyValue reference = MobUtils.createMobRefKeyValue(c, fileName, tableNameTag);
|
||||
// write the cell whose value is the path of a mob file to the store file.
|
||||
writer.append(reference);
|
||||
cellsCountCompactedToMob++;
|
||||
cellsSizeCompactedToMob += c.getValueLength();
|
||||
}
|
||||
++progress.currentCompactedKVs;
|
||||
// check periodically to see if a system stop is requested
|
||||
if (closeCheckInterval > 0) {
|
||||
bytesWritten += KeyValueUtil.length(c);
|
||||
if (bytesWritten > closeCheckInterval) {
|
||||
bytesWritten = 0;
|
||||
if (!store.areWritesEnabled()) {
|
||||
progress.cancel();
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
cells.clear();
|
||||
} while (hasMore);
|
||||
} finally {
|
||||
if (mobFileWriter != null) {
|
||||
mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells);
|
||||
mobFileWriter.close();
|
||||
}
|
||||
if (delFileWriter != null) {
|
||||
delFileWriter.appendMetadata(fd.maxSeqId, major, deleteMarkersCount);
|
||||
delFileWriter.close();
|
||||
}
|
||||
}
|
||||
if (mobFileWriter != null) {
|
||||
if (mobCells > 0) {
|
||||
// If the mob file is not empty, commit it.
|
||||
mobStore.commitFile(mobFileWriter.getPath(), path);
|
||||
} else {
|
||||
try {
|
||||
// If the mob file is empty, delete it instead of committing.
|
||||
store.getFileSystem().delete(mobFileWriter.getPath(), true);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to delete the temp mob file", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (delFileWriter != null) {
|
||||
if (deleteMarkersCount > 0) {
|
||||
// If the del file is not empty, commit it.
|
||||
// If the commit fails, the compaction is re-performed again.
|
||||
mobStore.commitFile(delFileWriter.getPath(), path);
|
||||
} else {
|
||||
try {
|
||||
// If the del file is empty, delete it instead of committing.
|
||||
store.getFileSystem().delete(delFileWriter.getPath(), true);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to delete the temp del file", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
|
||||
mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
|
||||
mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
|
||||
mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob);
|
||||
progress.complete();
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,226 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
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.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher;
|
||||
import org.apache.hadoop.hbase.regionserver.HMobStore;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
/**
|
||||
* An implementation of the StoreFlusher. It extends the DefaultStoreFlusher.
|
||||
* If the store is not a mob store, the flusher flushes the MemStore the same with
|
||||
* DefaultStoreFlusher,
|
||||
* If the store is a mob store, the flusher flushes the MemStore into two places.
|
||||
* One is the store files of HBase, the other is the mob files.
|
||||
* <ol>
|
||||
* <li>Cells that are not PUT type or have the delete mark will be directly flushed to HBase.</li>
|
||||
* <li>If the size of a cell value is larger than a threshold, it'll be flushed
|
||||
* to a mob file, another cell with the path of this file will be flushed to HBase.</li>
|
||||
* <li>If the size of a cell value is smaller than or equal with a threshold, it'll be flushed to
|
||||
* HBase directly.</li>
|
||||
* </ol>
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(DefaultMobStoreFlusher.class);
|
||||
private final Object flushLock = new Object();
|
||||
private long mobCellValueSizeThreshold = 0;
|
||||
private Path targetPath;
|
||||
private HMobStore mobStore;
|
||||
|
||||
public DefaultMobStoreFlusher(Configuration conf, Store store) throws IOException {
|
||||
super(conf, store);
|
||||
mobCellValueSizeThreshold = store.getFamily().getMobThreshold();
|
||||
this.targetPath = MobUtils.getMobFamilyPath(conf, store.getTableName(),
|
||||
store.getColumnFamilyName());
|
||||
if (!this.store.getFileSystem().exists(targetPath)) {
|
||||
this.store.getFileSystem().mkdirs(targetPath);
|
||||
}
|
||||
this.mobStore = (HMobStore) store;
|
||||
}
|
||||
|
||||
/**
|
||||
* Flushes the snapshot of the MemStore.
|
||||
* If this store is not a mob store, flush the cells in the snapshot to store files of HBase.
|
||||
* If the store is a mob one, the flusher flushes the MemStore into two places.
|
||||
* One is the store files of HBase, the other is the mob files.
|
||||
* <ol>
|
||||
* <li>Cells that are not PUT type or have the delete mark will be directly flushed to
|
||||
* HBase.</li>
|
||||
* <li>If the size of a cell value is larger than a threshold, it'll be
|
||||
* flushed to a mob file, another cell with the path of this file will be flushed to HBase.</li>
|
||||
* <li>If the size of a cell value is smaller than or equal with a threshold, it'll be flushed to
|
||||
* HBase directly.</li>
|
||||
* </ol>
|
||||
*/
|
||||
@Override
|
||||
public List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId,
|
||||
MonitoredTask status) throws IOException {
|
||||
ArrayList<Path> result = new ArrayList<Path>();
|
||||
int cellsCount = snapshot.getCellsCount();
|
||||
if (cellsCount == 0) return result; // don't flush if there are no entries
|
||||
|
||||
// Use a store scanner to find which rows to flush.
|
||||
long smallestReadPoint = store.getSmallestReadPoint();
|
||||
InternalScanner scanner = createScanner(snapshot.getScanner(), smallestReadPoint);
|
||||
if (scanner == null) {
|
||||
return result; // NULL scanner returned from coprocessor hooks means skip normal processing
|
||||
}
|
||||
StoreFile.Writer writer;
|
||||
try {
|
||||
// TODO: We can fail in the below block before we complete adding this flush to
|
||||
// list of store files. Add cleanup of anything put on filesystem if we fail.
|
||||
synchronized (flushLock) {
|
||||
status.setStatus("Flushing " + store + ": creating writer");
|
||||
// Write the map out to the disk
|
||||
writer = store.createWriterInTmp(cellsCount, store.getFamily().getCompression(),
|
||||
false, true, true);
|
||||
writer.setTimeRangeTracker(snapshot.getTimeRangeTracker());
|
||||
try {
|
||||
// It's a mob store, flush the cells in a mob way. This is the difference of flushing
|
||||
// between a normal and a mob store.
|
||||
performMobFlush(snapshot, cacheFlushId, scanner, writer, status);
|
||||
} finally {
|
||||
finalizeWriter(writer, cacheFlushId, status);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
scanner.close();
|
||||
}
|
||||
LOG.info("Mob store is flushed, sequenceid=" + cacheFlushId + ", memsize="
|
||||
+ StringUtils.TraditionalBinaryPrefix.long2String(snapshot.getSize(), "", 1) +
|
||||
", hasBloomFilter=" + writer.hasGeneralBloom() +
|
||||
", into tmp file " + writer.getPath());
|
||||
result.add(writer.getPath());
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Flushes the cells in the mob store.
|
||||
* <ol>In the mob store, the cells with PUT type might have or have no mob tags.
|
||||
* <li>If a cell does not have a mob tag, flushing the cell to different files depends
|
||||
* on the value length. If the length is larger than a threshold, it's flushed to a
|
||||
* mob file and the mob file is flushed to a store file in HBase. Otherwise, directly
|
||||
* flush the cell to a store file in HBase.</li>
|
||||
* <li>If a cell have a mob tag, its value is a mob file name, directly flush it
|
||||
* to a store file in HBase.</li>
|
||||
* </ol>
|
||||
* @param snapshot Memstore snapshot.
|
||||
* @param cacheFlushId Log cache flush sequence number.
|
||||
* @param scanner The scanner of memstore snapshot.
|
||||
* @param writer The store file writer.
|
||||
* @param status Task that represents the flush operation and may be updated with status.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId,
|
||||
InternalScanner scanner, StoreFile.Writer writer, MonitoredTask status) throws IOException {
|
||||
StoreFile.Writer mobFileWriter = null;
|
||||
int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX,
|
||||
HConstants.COMPACTION_KV_MAX_DEFAULT);
|
||||
long mobCount = 0;
|
||||
long mobSize = 0;
|
||||
long time = snapshot.getTimeRangeTracker().getMaximumTimestamp();
|
||||
mobFileWriter = mobStore.createWriterInTmp(new Date(time), snapshot.getCellsCount(),
|
||||
store.getFamily().getCompression(), store.getRegionInfo().getStartKey());
|
||||
// the target path is {tableName}/.mob/{cfName}/mobFiles
|
||||
// the relative path is mobFiles
|
||||
byte[] fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
|
||||
try {
|
||||
Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, store.getTableName()
|
||||
.getName());
|
||||
List<Cell> cells = new ArrayList<Cell>();
|
||||
boolean hasMore;
|
||||
ScannerContext scannerContext =
|
||||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
||||
|
||||
do {
|
||||
hasMore = scanner.next(cells, scannerContext);
|
||||
if (!cells.isEmpty()) {
|
||||
for (Cell c : cells) {
|
||||
// If we know that this KV is going to be included always, then let us
|
||||
// set its memstoreTS to 0. This will help us save space when writing to
|
||||
// disk.
|
||||
if (c.getValueLength() <= mobCellValueSizeThreshold || MobUtils.isMobReferenceCell(c)
|
||||
|| c.getTypeByte() != KeyValue.Type.Put.getCode()) {
|
||||
writer.append(c);
|
||||
} else {
|
||||
// append the original keyValue in the mob file.
|
||||
mobFileWriter.append(c);
|
||||
mobSize += c.getValueLength();
|
||||
mobCount++;
|
||||
|
||||
// append the tags to the KeyValue.
|
||||
// The key is same, the value is the filename of the mob file
|
||||
KeyValue reference = MobUtils.createMobRefKeyValue(c, fileName, tableNameTag);
|
||||
writer.append(reference);
|
||||
}
|
||||
}
|
||||
cells.clear();
|
||||
}
|
||||
} while (hasMore);
|
||||
} finally {
|
||||
status.setStatus("Flushing mob file " + store + ": appending metadata");
|
||||
mobFileWriter.appendMetadata(cacheFlushId, false, mobCount);
|
||||
status.setStatus("Flushing mob file " + store + ": closing flushed file");
|
||||
mobFileWriter.close();
|
||||
}
|
||||
|
||||
if (mobCount > 0) {
|
||||
// commit the mob file from temp folder to target folder.
|
||||
// If the mob file is committed successfully but the store file is not,
|
||||
// the committed mob file will be handled by the sweep tool as an unused
|
||||
// file.
|
||||
mobStore.commitFile(mobFileWriter.getPath(), targetPath);
|
||||
mobStore.updateMobFlushCount();
|
||||
mobStore.updateMobFlushedCellsCount(mobCount);
|
||||
mobStore.updateMobFlushedCellsSize(mobSize);
|
||||
} else {
|
||||
try {
|
||||
// If the mob file is empty, delete it instead of committing.
|
||||
store.getFileSystem().delete(mobFileWriter.getPath(), true);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to delete the temp mob file", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,120 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* The cleaner to delete the expired MOB files.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ExpiredMobFileCleaner extends Configured implements Tool {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ExpiredMobFileCleaner.class);
|
||||
/**
|
||||
* Cleans the MOB files when they're expired and their min versions are 0.
|
||||
* If the latest timestamp of Cells in a MOB file is older than the TTL in the column family,
|
||||
* it's regarded as expired. This cleaner deletes them.
|
||||
* At a time T0, the cells in a mob file M0 are expired. If a user starts a scan before T0, those
|
||||
* mob cells are visible, this scan still runs after T0. At that time T1, this mob file M0
|
||||
* is expired, meanwhile a cleaner starts, the M0 is archived and can be read in the archive
|
||||
* directory.
|
||||
* @param tableName The current table name.
|
||||
* @param family The current family.
|
||||
* @throws ServiceException
|
||||
* @throws IOException
|
||||
*/
|
||||
public void cleanExpiredMobFiles(String tableName, HColumnDescriptor family)
|
||||
throws ServiceException, IOException {
|
||||
Configuration conf = getConf();
|
||||
TableName tn = TableName.valueOf(tableName);
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
LOG.info("Cleaning the expired MOB files of " + family.getNameAsString() + " in " + tableName);
|
||||
// disable the block cache.
|
||||
Configuration copyOfConf = new Configuration(conf);
|
||||
copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
|
||||
CacheConfig cacheConfig = new CacheConfig(copyOfConf);
|
||||
MobUtils.cleanExpiredMobFiles(fs, conf, tn, family, cacheConfig,
|
||||
EnvironmentEdgeManager.currentTime());
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
ToolRunner.run(conf, new ExpiredMobFileCleaner(), args);
|
||||
}
|
||||
|
||||
private void printUsage() {
|
||||
System.err.println("Usage:\n" + "--------------------------\n"
|
||||
+ ExpiredMobFileCleaner.class.getName() + " tableName familyName");
|
||||
System.err.println(" tableName The table name");
|
||||
System.err.println(" familyName The column family name");
|
||||
}
|
||||
|
||||
public int run(String[] args) throws Exception {
|
||||
if (args.length != 2) {
|
||||
printUsage();
|
||||
return 1;
|
||||
}
|
||||
String tableName = args[0];
|
||||
String familyName = args[1];
|
||||
TableName tn = TableName.valueOf(tableName);
|
||||
HBaseAdmin.checkHBaseAvailable(getConf());
|
||||
HBaseAdmin admin = new HBaseAdmin(getConf());
|
||||
try {
|
||||
HTableDescriptor htd = admin.getTableDescriptor(tn);
|
||||
HColumnDescriptor family = htd.getFamily(Bytes.toBytes(familyName));
|
||||
if (family == null || !family.isMobEnabled()) {
|
||||
throw new IOException("Column family " + familyName + " is not a MOB column family");
|
||||
}
|
||||
if (family.getMinVersions() > 0) {
|
||||
throw new IOException(
|
||||
"The minVersions of the column family is not 0, could not be handled by this cleaner");
|
||||
}
|
||||
cleanExpiredMobFiles(tableName, family);
|
||||
return 0;
|
||||
} finally {
|
||||
try {
|
||||
admin.close();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to close the HBaseAdmin.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
||||
/**
|
||||
* The cache configuration for the mob.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobCacheConfig extends CacheConfig {
|
||||
|
||||
private static MobFileCache mobFileCache;
|
||||
|
||||
public MobCacheConfig(Configuration conf, HColumnDescriptor family) {
|
||||
super(conf, family);
|
||||
instantiateMobFileCache(conf);
|
||||
}
|
||||
|
||||
public MobCacheConfig(Configuration conf) {
|
||||
super(conf);
|
||||
instantiateMobFileCache(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Instantiates the MobFileCache.
|
||||
* @param conf The current configuration.
|
||||
* @return The current instance of MobFileCache.
|
||||
*/
|
||||
public static synchronized MobFileCache instantiateMobFileCache(Configuration conf) {
|
||||
if (mobFileCache == null) {
|
||||
mobFileCache = new MobFileCache(conf);
|
||||
}
|
||||
return mobFileCache;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the MobFileCache.
|
||||
* @return The MobFileCache.
|
||||
*/
|
||||
public MobFileCache getMobFileCache() {
|
||||
return mobFileCache;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,121 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* The constants used in mob.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public final class MobConstants {
|
||||
|
||||
public static final String MOB_SCAN_RAW = "hbase.mob.scan.raw";
|
||||
public static final String MOB_CACHE_BLOCKS = "hbase.mob.cache.blocks";
|
||||
public static final String MOB_SCAN_REF_ONLY = "hbase.mob.scan.ref.only";
|
||||
public static final String EMPTY_VALUE_ON_MOBCELL_MISS = "empty.value.on.mobcell.miss";
|
||||
|
||||
public static final String MOB_FILE_CACHE_SIZE_KEY = "hbase.mob.file.cache.size";
|
||||
public static final int DEFAULT_MOB_FILE_CACHE_SIZE = 1000;
|
||||
|
||||
public static final String MOB_DIR_NAME = "mobdir";
|
||||
public static final String MOB_REGION_NAME = ".mob";
|
||||
public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME);
|
||||
|
||||
public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.ttl.cleaner.period";
|
||||
public static final int DEFAULT_MOB_CLEANER_PERIOD = 24 * 60 * 60; // one day
|
||||
|
||||
public static final String MOB_SWEEP_TOOL_COMPACTION_START_DATE =
|
||||
"hbase.mob.sweep.tool.compaction.start.date";
|
||||
public static final String MOB_SWEEP_TOOL_COMPACTION_RATIO =
|
||||
"hbase.mob.sweep.tool.compaction.ratio";
|
||||
public static final String MOB_SWEEP_TOOL_COMPACTION_MERGEABLE_SIZE =
|
||||
"hbase.mob.sweep.tool.compaction.mergeable.size";
|
||||
|
||||
public static final float DEFAULT_SWEEP_TOOL_MOB_COMPACTION_RATIO = 0.5f;
|
||||
public static final long DEFAULT_SWEEP_TOOL_MOB_COMPACTION_MERGEABLE_SIZE = 128 * 1024 * 1024;
|
||||
|
||||
public static final String MOB_SWEEP_TOOL_COMPACTION_TEMP_DIR_NAME = "mobcompaction";
|
||||
|
||||
public static final String MOB_SWEEP_TOOL_COMPACTION_MEMSTORE_FLUSH_SIZE =
|
||||
"hbase.mob.sweep.tool.compaction.memstore.flush.size";
|
||||
public static final long DEFAULT_MOB_SWEEP_TOOL_COMPACTION_MEMSTORE_FLUSH_SIZE =
|
||||
1024 * 1024 * 128; // 128M
|
||||
|
||||
public static final String MOB_CACHE_EVICT_PERIOD = "hbase.mob.cache.evict.period";
|
||||
public static final String MOB_CACHE_EVICT_REMAIN_RATIO = "hbase.mob.cache.evict.remain.ratio";
|
||||
public static final Tag MOB_REF_TAG = new Tag(TagType.MOB_REFERENCE_TAG_TYPE,
|
||||
HConstants.EMPTY_BYTE_ARRAY);
|
||||
|
||||
public static final float DEFAULT_EVICT_REMAIN_RATIO = 0.5f;
|
||||
public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L;
|
||||
|
||||
public final static String TEMP_DIR_NAME = ".tmp";
|
||||
public final static String BULKLOAD_DIR_NAME = ".bulkload";
|
||||
public final static byte[] MOB_TABLE_LOCK_SUFFIX = Bytes.toBytes(".mobLock");
|
||||
public final static String EMPTY_STRING = "";
|
||||
/**
|
||||
* If the size of a mob file is less than this value, it's regarded as a small file and needs to
|
||||
* be merged in mob compaction. The default value is 192MB.
|
||||
*/
|
||||
public static final String MOB_COMPACTION_MERGEABLE_THRESHOLD =
|
||||
"hbase.mob.compaction.mergeable.threshold";
|
||||
public static final long DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD = 192 * 1024 * 1024;
|
||||
/**
|
||||
* The max number of del files that is allowed in the mob file compaction. In the mob
|
||||
* compaction, when the number of existing del files is larger than this value, they are merged
|
||||
* until number of del files is not larger this value. The default value is 3.
|
||||
*/
|
||||
public static final String MOB_DELFILE_MAX_COUNT = "hbase.mob.delfile.max.count";
|
||||
public static final int DEFAULT_MOB_DELFILE_MAX_COUNT = 3;
|
||||
/**
|
||||
* The max number of the mob files that is allowed in a batch of the mob compaction.
|
||||
* The mob compaction merges the small mob files to bigger ones. If the number of the
|
||||
* small files is very large, it could lead to a "too many opened file handlers" in the merge.
|
||||
* And the merge has to be split into batches. This value limits the number of mob files
|
||||
* that are selected in a batch of the mob compaction. The default value is 100.
|
||||
*/
|
||||
public static final String MOB_COMPACTION_BATCH_SIZE =
|
||||
"hbase.mob.compaction.batch.size";
|
||||
public static final int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100;
|
||||
/**
|
||||
* The period that MobCompactionChore runs. The unit is second.
|
||||
* The default value is one week.
|
||||
*/
|
||||
public static final String MOB_COMPACTION_CHORE_PERIOD =
|
||||
"hbase.mob.compaction.chore.period";
|
||||
public static final int DEFAULT_MOB_COMPACTION_CHORE_PERIOD =
|
||||
24 * 60 * 60 * 7; // a week
|
||||
public static final String MOB_COMPACTOR_CLASS_KEY = "hbase.mob.compactor.class";
|
||||
/**
|
||||
* The max number of threads used in MobCompactor.
|
||||
*/
|
||||
public static final String MOB_COMPACTION_THREADS_MAX =
|
||||
"hbase.mob.compaction.threads.max";
|
||||
public static final int DEFAULT_MOB_COMPACTION_THREADS_MAX = 1;
|
||||
private MobConstants() {
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,152 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
|
||||
/**
|
||||
* The mob file.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobFile {
|
||||
|
||||
private StoreFile sf;
|
||||
|
||||
// internal use only for sub classes
|
||||
protected MobFile() {
|
||||
}
|
||||
|
||||
protected MobFile(StoreFile sf) {
|
||||
this.sf = sf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal use only. This is used by the sweeper.
|
||||
*
|
||||
* @return The store file scanner.
|
||||
* @throws IOException
|
||||
*/
|
||||
public StoreFileScanner getScanner() throws IOException {
|
||||
List<StoreFile> sfs = new ArrayList<StoreFile>();
|
||||
sfs.add(sf);
|
||||
List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true,
|
||||
false, null, sf.getMaxMemstoreTS());
|
||||
|
||||
return sfScanners.get(0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a cell from the mob file.
|
||||
* @param search The cell need to be searched in the mob file.
|
||||
* @param cacheMobBlocks Should this scanner cache blocks.
|
||||
* @return The cell in the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Cell readCell(Cell search, boolean cacheMobBlocks) throws IOException {
|
||||
return readCell(search, cacheMobBlocks, sf.getMaxMemstoreTS());
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a cell from the mob file.
|
||||
* @param search The cell need to be searched in the mob file.
|
||||
* @param cacheMobBlocks Should this scanner cache blocks.
|
||||
* @param readPt the read point.
|
||||
* @return The cell in the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Cell readCell(Cell search, boolean cacheMobBlocks, long readPt) throws IOException {
|
||||
Cell result = null;
|
||||
StoreFileScanner scanner = null;
|
||||
List<StoreFile> sfs = new ArrayList<StoreFile>();
|
||||
sfs.add(sf);
|
||||
try {
|
||||
List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(sfs,
|
||||
cacheMobBlocks, true, false, null, readPt);
|
||||
if (!sfScanners.isEmpty()) {
|
||||
scanner = sfScanners.get(0);
|
||||
if (scanner.seek(search)) {
|
||||
result = scanner.peek();
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (scanner != null) {
|
||||
scanner.close();
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the file name.
|
||||
* @return The file name.
|
||||
*/
|
||||
public String getFileName() {
|
||||
return sf.getPath().getName();
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens the underlying reader.
|
||||
* It's not thread-safe. Use MobFileCache.openFile() instead.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void open() throws IOException {
|
||||
if (sf.getReader() == null) {
|
||||
sf.createReader();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the underlying reader, but do no evict blocks belonging to this file.
|
||||
* It's not thread-safe. Use MobFileCache.closeFile() instead.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
if (sf != null) {
|
||||
sf.closeReader(false);
|
||||
sf = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an instance of the MobFile.
|
||||
* @param fs The file system.
|
||||
* @param path The path of the underlying StoreFile.
|
||||
* @param conf The configuration.
|
||||
* @param cacheConf The CacheConfig.
|
||||
* @return An instance of the MobFile.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static MobFile create(FileSystem fs, Path path, Configuration conf, CacheConfig cacheConf)
|
||||
throws IOException {
|
||||
StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE);
|
||||
return new MobFile(sf);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,325 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.IdLock;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* The cache for mob files.
|
||||
* This cache doesn't cache the mob file blocks. It only caches the references of mob files.
|
||||
* We are doing this to avoid opening and closing mob files all the time. We just keep
|
||||
* references open.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobFileCache {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(MobFileCache.class);
|
||||
|
||||
/*
|
||||
* Eviction and statistics thread. Periodically run to print the statistics and
|
||||
* evict the lru cached mob files when the count of the cached files is larger
|
||||
* than the threshold.
|
||||
*/
|
||||
static class EvictionThread extends Thread {
|
||||
MobFileCache lru;
|
||||
|
||||
public EvictionThread(MobFileCache lru) {
|
||||
super("MobFileCache.EvictionThread");
|
||||
setDaemon(true);
|
||||
this.lru = lru;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
lru.evict();
|
||||
}
|
||||
}
|
||||
|
||||
// a ConcurrentHashMap, accesses to this map are synchronized.
|
||||
private Map<String, CachedMobFile> map = null;
|
||||
// caches access count
|
||||
private final AtomicLong count = new AtomicLong(0);
|
||||
private long lastAccess = 0;
|
||||
private final AtomicLong miss = new AtomicLong(0);
|
||||
private long lastMiss = 0;
|
||||
private final AtomicLong evictedFileCount = new AtomicLong(0);
|
||||
private long lastEvictedFileCount = 0;
|
||||
|
||||
// a lock to sync the evict to guarantee the eviction occurs in sequence.
|
||||
// the method evictFile is not sync by this lock, the ConcurrentHashMap does the sync there.
|
||||
private final ReentrantLock evictionLock = new ReentrantLock(true);
|
||||
|
||||
//stripes lock on each mob file based on its hash. Sync the openFile/closeFile operations.
|
||||
private final IdLock keyLock = new IdLock();
|
||||
|
||||
private final ScheduledExecutorService scheduleThreadPool = Executors.newScheduledThreadPool(1,
|
||||
new ThreadFactoryBuilder().setNameFormat("MobFileCache #%d").setDaemon(true).build());
|
||||
private final Configuration conf;
|
||||
|
||||
// the count of the cached references to mob files
|
||||
private final int mobFileMaxCacheSize;
|
||||
private final boolean isCacheEnabled;
|
||||
private float evictRemainRatio;
|
||||
|
||||
public MobFileCache(Configuration conf) {
|
||||
this.conf = conf;
|
||||
this.mobFileMaxCacheSize = conf.getInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY,
|
||||
MobConstants.DEFAULT_MOB_FILE_CACHE_SIZE);
|
||||
isCacheEnabled = (mobFileMaxCacheSize > 0);
|
||||
map = new ConcurrentHashMap<String, CachedMobFile>(mobFileMaxCacheSize);
|
||||
if (isCacheEnabled) {
|
||||
long period = conf.getLong(MobConstants.MOB_CACHE_EVICT_PERIOD,
|
||||
MobConstants.DEFAULT_MOB_CACHE_EVICT_PERIOD); // in seconds
|
||||
evictRemainRatio = conf.getFloat(MobConstants.MOB_CACHE_EVICT_REMAIN_RATIO,
|
||||
MobConstants.DEFAULT_EVICT_REMAIN_RATIO);
|
||||
if (evictRemainRatio < 0.0) {
|
||||
evictRemainRatio = 0.0f;
|
||||
LOG.warn(MobConstants.MOB_CACHE_EVICT_REMAIN_RATIO + " is less than 0.0, 0.0 is used.");
|
||||
} else if (evictRemainRatio > 1.0) {
|
||||
evictRemainRatio = 1.0f;
|
||||
LOG.warn(MobConstants.MOB_CACHE_EVICT_REMAIN_RATIO + " is larger than 1.0, 1.0 is used.");
|
||||
}
|
||||
this.scheduleThreadPool.scheduleAtFixedRate(new EvictionThread(this), period, period,
|
||||
TimeUnit.SECONDS);
|
||||
}
|
||||
LOG.info("MobFileCache is initialized, and the cache size is " + mobFileMaxCacheSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Evicts the lru cached mob files when the count of the cached files is larger
|
||||
* than the threshold.
|
||||
*/
|
||||
public void evict() {
|
||||
if (isCacheEnabled) {
|
||||
// Ensure only one eviction at a time
|
||||
if (!evictionLock.tryLock()) {
|
||||
return;
|
||||
}
|
||||
printStatistics();
|
||||
List<CachedMobFile> evictedFiles = new ArrayList<CachedMobFile>();
|
||||
try {
|
||||
if (map.size() <= mobFileMaxCacheSize) {
|
||||
return;
|
||||
}
|
||||
List<CachedMobFile> files = new ArrayList<CachedMobFile>(map.values());
|
||||
Collections.sort(files);
|
||||
int start = (int) (mobFileMaxCacheSize * evictRemainRatio);
|
||||
if (start >= 0) {
|
||||
for (int i = start; i < files.size(); i++) {
|
||||
String name = files.get(i).getFileName();
|
||||
CachedMobFile evictedFile = map.remove(name);
|
||||
if (evictedFile != null) {
|
||||
evictedFiles.add(evictedFile);
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
evictionLock.unlock();
|
||||
}
|
||||
// EvictionLock is released. Close the evicted files one by one.
|
||||
// The closes are sync in the closeFile method.
|
||||
for (CachedMobFile evictedFile : evictedFiles) {
|
||||
closeFile(evictedFile);
|
||||
}
|
||||
evictedFileCount.addAndGet(evictedFiles.size());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Evicts the cached file by the name.
|
||||
* @param fileName The name of a cached file.
|
||||
*/
|
||||
public void evictFile(String fileName) {
|
||||
if (isCacheEnabled) {
|
||||
IdLock.Entry lockEntry = null;
|
||||
try {
|
||||
// obtains the lock to close the cached file.
|
||||
lockEntry = keyLock.getLockEntry(fileName.hashCode());
|
||||
CachedMobFile evictedFile = map.remove(fileName);
|
||||
if (evictedFile != null) {
|
||||
evictedFile.close();
|
||||
evictedFileCount.incrementAndGet();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to evict the file " + fileName, e);
|
||||
} finally {
|
||||
if (lockEntry != null) {
|
||||
keyLock.releaseLockEntry(lockEntry);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens a mob file.
|
||||
* @param fs The current file system.
|
||||
* @param path The file path.
|
||||
* @param cacheConf The current MobCacheConfig
|
||||
* @return A opened mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public MobFile openFile(FileSystem fs, Path path, MobCacheConfig cacheConf) throws IOException {
|
||||
if (!isCacheEnabled) {
|
||||
MobFile mobFile = MobFile.create(fs, path, conf, cacheConf);
|
||||
mobFile.open();
|
||||
return mobFile;
|
||||
} else {
|
||||
String fileName = path.getName();
|
||||
CachedMobFile cached = map.get(fileName);
|
||||
IdLock.Entry lockEntry = keyLock.getLockEntry(fileName.hashCode());
|
||||
try {
|
||||
if (cached == null) {
|
||||
cached = map.get(fileName);
|
||||
if (cached == null) {
|
||||
if (map.size() > mobFileMaxCacheSize) {
|
||||
evict();
|
||||
}
|
||||
cached = CachedMobFile.create(fs, path, conf, cacheConf);
|
||||
cached.open();
|
||||
map.put(fileName, cached);
|
||||
miss.incrementAndGet();
|
||||
}
|
||||
}
|
||||
cached.open();
|
||||
cached.access(count.incrementAndGet());
|
||||
} finally {
|
||||
keyLock.releaseLockEntry(lockEntry);
|
||||
}
|
||||
return cached;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes a mob file.
|
||||
* @param file The mob file that needs to be closed.
|
||||
*/
|
||||
public void closeFile(MobFile file) {
|
||||
IdLock.Entry lockEntry = null;
|
||||
try {
|
||||
if (!isCacheEnabled) {
|
||||
file.close();
|
||||
} else {
|
||||
lockEntry = keyLock.getLockEntry(file.getFileName().hashCode());
|
||||
file.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("MobFileCache, Exception happen during close " + file.getFileName(), e);
|
||||
} finally {
|
||||
if (lockEntry != null) {
|
||||
keyLock.releaseLockEntry(lockEntry);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void shutdown() {
|
||||
this.scheduleThreadPool.shutdown();
|
||||
for (int i = 0; i < 100; i++) {
|
||||
if (!this.scheduleThreadPool.isShutdown()) {
|
||||
try {
|
||||
Thread.sleep(10);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("Interrupted while sleeping");
|
||||
Thread.currentThread().interrupt();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!this.scheduleThreadPool.isShutdown()) {
|
||||
List<Runnable> runnables = this.scheduleThreadPool.shutdownNow();
|
||||
LOG.debug("Still running " + runnables);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the count of cached mob files.
|
||||
* @return The count of the cached mob files.
|
||||
*/
|
||||
public int getCacheSize() {
|
||||
return map == null ? 0 : map.size();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the count of accesses to the mob file cache.
|
||||
* @return The count of accesses to the mob file cache.
|
||||
*/
|
||||
public long getAccessCount() {
|
||||
return count.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the count of misses to the mob file cache.
|
||||
* @return The count of misses to the mob file cache.
|
||||
*/
|
||||
public long getMissCount() {
|
||||
return miss.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of items evicted from the mob file cache.
|
||||
* @return The number of items evicted from the mob file cache.
|
||||
*/
|
||||
public long getEvictedFileCount() {
|
||||
return evictedFileCount.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the hit ratio to the mob file cache.
|
||||
* @return The hit ratio to the mob file cache.
|
||||
*/
|
||||
public double getHitRatio() {
|
||||
return count.get() == 0 ? 0 : ((float) (count.get() - miss.get())) / (float) count.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Prints the statistics.
|
||||
*/
|
||||
public void printStatistics() {
|
||||
long access = count.get() - lastAccess;
|
||||
long missed = miss.get() - lastMiss;
|
||||
long evicted = evictedFileCount.get() - lastEvictedFileCount;
|
||||
int hitRatio = access == 0 ? 0 : (int) (((float) (access - missed)) / (float) access * 100);
|
||||
LOG.info("MobFileCache Statistics, access: " + access + ", miss: " + missed + ", hit: "
|
||||
+ (access - missed) + ", hit ratio: " + hitRatio + "%, evicted files: " + evicted);
|
||||
lastAccess += access;
|
||||
lastMiss += missed;
|
||||
lastEvictedFileCount += evicted;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,169 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.MD5Hash;
|
||||
|
||||
/**
|
||||
* The mob file name.
|
||||
* It consists of a md5 of a start key, a date and an uuid.
|
||||
* It looks like md5(start) + date + uuid.
|
||||
* <ol>
|
||||
* <li>characters 0-31: md5 hex string of a start key. Since the length of the start key is not
|
||||
* fixed, have to use the md5 instead which has a fix length.</li>
|
||||
* <li>characters 32-39: a string of a date with format yyyymmdd. The date is the latest timestamp
|
||||
* of cells in this file</li>
|
||||
* <li>the remaining characters: the uuid.</li>
|
||||
* </ol>
|
||||
* Using md5 hex string of start key as the prefix of file name makes files with the same start
|
||||
* key unique, they're different from the ones with other start keys
|
||||
* The cells come from different regions might be in the same mob file by region split,
|
||||
* this is allowed.
|
||||
* Has the latest timestamp of cells in the file name in order to clean the expired mob files by
|
||||
* TTL easily. If this timestamp is older than the TTL, it's regarded as expired.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public final class MobFileName {
|
||||
|
||||
private final String date;
|
||||
private final String startKey;
|
||||
private final String uuid;
|
||||
private final String fileName;
|
||||
|
||||
/**
|
||||
* @param startKey
|
||||
* The start key.
|
||||
* @param date
|
||||
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
||||
* @param uuid
|
||||
* The uuid
|
||||
*/
|
||||
private MobFileName(byte[] startKey, String date, String uuid) {
|
||||
this.startKey = MD5Hash.getMD5AsHex(startKey, 0, startKey.length);
|
||||
this.uuid = uuid;
|
||||
this.date = date;
|
||||
this.fileName = this.startKey + date + uuid;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param startKey
|
||||
* The md5 hex string of the start key.
|
||||
* @param date
|
||||
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
||||
* @param uuid
|
||||
* The uuid
|
||||
*/
|
||||
private MobFileName(String startKey, String date, String uuid) {
|
||||
this.startKey = startKey;
|
||||
this.uuid = uuid;
|
||||
this.date = date;
|
||||
this.fileName = this.startKey + date + uuid;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an instance of MobFileName
|
||||
*
|
||||
* @param startKey
|
||||
* The start key.
|
||||
* @param date
|
||||
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
||||
* @param uuid The uuid.
|
||||
* @return An instance of a MobFileName.
|
||||
*/
|
||||
public static MobFileName create(byte[] startKey, String date, String uuid) {
|
||||
return new MobFileName(startKey, date, uuid);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an instance of MobFileName
|
||||
*
|
||||
* @param startKey
|
||||
* The md5 hex string of the start key.
|
||||
* @param date
|
||||
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
||||
* @param uuid The uuid.
|
||||
* @return An instance of a MobFileName.
|
||||
*/
|
||||
public static MobFileName create(String startKey, String date, String uuid) {
|
||||
return new MobFileName(startKey, date, uuid);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an instance of MobFileName.
|
||||
* @param fileName The string format of a file name.
|
||||
* @return An instance of a MobFileName.
|
||||
*/
|
||||
public static MobFileName create(String fileName) {
|
||||
// The format of a file name is md5HexString(0-31bytes) + date(32-39bytes) + UUID
|
||||
// The date format is yyyyMMdd
|
||||
String startKey = fileName.substring(0, 32);
|
||||
String date = fileName.substring(32, 40);
|
||||
String uuid = fileName.substring(40);
|
||||
return new MobFileName(startKey, date, uuid);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the hex string of the md5 for a start key.
|
||||
* @return The hex string of the md5 for a start key.
|
||||
*/
|
||||
public String getStartKey() {
|
||||
return startKey;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the date string. Its format is yyyymmdd.
|
||||
* @return The date string.
|
||||
*/
|
||||
public String getDate() {
|
||||
return this.date;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append(startKey);
|
||||
builder.append(date);
|
||||
builder.append(uuid);
|
||||
return builder.toString().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object anObject) {
|
||||
if (this == anObject) {
|
||||
return true;
|
||||
}
|
||||
if (anObject instanceof MobFileName) {
|
||||
MobFileName another = (MobFileName) anObject;
|
||||
if (this.startKey.equals(another.startKey) && this.date.equals(another.date)
|
||||
&& this.uuid.equals(another.uuid)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the file name.
|
||||
* @return The file name.
|
||||
*/
|
||||
public String getFileName() {
|
||||
return this.fileName;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/**
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
|
||||
/**
|
||||
* MobStoreEngine creates the mob specific compactor, and store flusher.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobStoreEngine extends DefaultStoreEngine {
|
||||
|
||||
@Override
|
||||
protected void createStoreFlusher(Configuration conf, Store store) throws IOException {
|
||||
// When using MOB, we use DefaultMobStoreFlusher always
|
||||
// Just use the compactor and compaction policy as that in DefaultStoreEngine. We can have MOB
|
||||
// specific compactor and policy when that is implemented.
|
||||
storeFlusher = new DefaultMobStoreFlusher(conf, store);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the DefaultMobCompactor.
|
||||
*/
|
||||
@Override
|
||||
protected void createCompactor(Configuration conf, Store store) throws IOException {
|
||||
compactor = new DefaultMobStoreCompactor(conf, store);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,898 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.security.Key;
|
||||
import java.security.KeyException;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.RejectedExecutionHandler;
|
||||
import java.util.concurrent.SynchronousQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
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.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.crypto.Cipher;
|
||||
import org.apache.hadoop.hbase.io.crypto.Encryption;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.mob.compactions.MobCompactor;
|
||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.security.EncryptionUtil;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
||||
/**
|
||||
* The mob utilities
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobUtils {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(MobUtils.class);
|
||||
|
||||
private static final ThreadLocal<SimpleDateFormat> LOCAL_FORMAT =
|
||||
new ThreadLocal<SimpleDateFormat>() {
|
||||
@Override
|
||||
protected SimpleDateFormat initialValue() {
|
||||
return new SimpleDateFormat("yyyyMMdd");
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Formats a date to a string.
|
||||
* @param date The date.
|
||||
* @return The string format of the date, it's yyyymmdd.
|
||||
*/
|
||||
public static String formatDate(Date date) {
|
||||
return LOCAL_FORMAT.get().format(date);
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses the string to a date.
|
||||
* @param dateString The string format of a date, it's yyyymmdd.
|
||||
* @return A date.
|
||||
* @throws ParseException
|
||||
*/
|
||||
public static Date parseDate(String dateString) throws ParseException {
|
||||
return LOCAL_FORMAT.get().parse(dateString);
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether the current cell is a mob reference cell.
|
||||
* @param cell The current cell.
|
||||
* @return True if the cell has a mob reference tag, false if it doesn't.
|
||||
*/
|
||||
public static boolean isMobReferenceCell(Cell cell) {
|
||||
if (cell.getTagsLength() > 0) {
|
||||
Tag tag = Tag.getTag(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength(),
|
||||
TagType.MOB_REFERENCE_TAG_TYPE);
|
||||
return tag != null;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the table name tag.
|
||||
* @param cell The current cell.
|
||||
* @return The table name tag.
|
||||
*/
|
||||
public static Tag getTableNameTag(Cell cell) {
|
||||
if (cell.getTagsLength() > 0) {
|
||||
Tag tag = Tag.getTag(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength(),
|
||||
TagType.MOB_TABLE_NAME_TAG_TYPE);
|
||||
return tag;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether the tag list has a mob reference tag.
|
||||
* @param tags The tag list.
|
||||
* @return True if the list has a mob reference tag, false if it doesn't.
|
||||
*/
|
||||
public static boolean hasMobReferenceTag(List<Tag> tags) {
|
||||
if (!tags.isEmpty()) {
|
||||
for (Tag tag : tags) {
|
||||
if (tag.getType() == TagType.MOB_REFERENCE_TAG_TYPE) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether it's a raw scan.
|
||||
* The information is set in the attribute "hbase.mob.scan.raw" of scan.
|
||||
* For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob file.
|
||||
* In a raw scan, the scanner directly returns cell in HBase without retrieve the one in
|
||||
* the mob file.
|
||||
* @param scan The current scan.
|
||||
* @return True if it's a raw scan.
|
||||
*/
|
||||
public static boolean isRawMobScan(Scan scan) {
|
||||
byte[] raw = scan.getAttribute(MobConstants.MOB_SCAN_RAW);
|
||||
try {
|
||||
return raw != null && Bytes.toBoolean(raw);
|
||||
} catch (IllegalArgumentException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether it's a reference only scan.
|
||||
* The information is set in the attribute "hbase.mob.scan.ref.only" of scan.
|
||||
* If it's a ref only scan, only the cells with ref tag are returned.
|
||||
* @param scan The current scan.
|
||||
* @return True if it's a ref only scan.
|
||||
*/
|
||||
public static boolean isRefOnlyScan(Scan scan) {
|
||||
byte[] refOnly = scan.getAttribute(MobConstants.MOB_SCAN_REF_ONLY);
|
||||
try {
|
||||
return refOnly != null && Bytes.toBoolean(refOnly);
|
||||
} catch (IllegalArgumentException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the scan contains the information of caching blocks.
|
||||
* The information is set in the attribute "hbase.mob.cache.blocks" of scan.
|
||||
* @param scan The current scan.
|
||||
* @return True when the Scan attribute specifies to cache the MOB blocks.
|
||||
*/
|
||||
public static boolean isCacheMobBlocks(Scan scan) {
|
||||
byte[] cache = scan.getAttribute(MobConstants.MOB_CACHE_BLOCKS);
|
||||
try {
|
||||
return cache != null && Bytes.toBoolean(cache);
|
||||
} catch (IllegalArgumentException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the attribute of caching blocks in the scan.
|
||||
*
|
||||
* @param scan
|
||||
* The current scan.
|
||||
* @param cacheBlocks
|
||||
* True, set the attribute of caching blocks into the scan, the scanner with this scan
|
||||
* caches blocks.
|
||||
* False, the scanner doesn't cache blocks for this scan.
|
||||
*/
|
||||
public static void setCacheMobBlocks(Scan scan, boolean cacheBlocks) {
|
||||
scan.setAttribute(MobConstants.MOB_CACHE_BLOCKS, Bytes.toBytes(cacheBlocks));
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleans the expired mob files.
|
||||
* Cleans the files whose creation date is older than (current - columnFamily.ttl), and
|
||||
* the minVersions of that column family is 0.
|
||||
* @param fs The current file system.
|
||||
* @param conf The current configuration.
|
||||
* @param tableName The current table name.
|
||||
* @param columnDescriptor The descriptor of the current column family.
|
||||
* @param cacheConfig The cacheConfig that disables the block cache.
|
||||
* @param current The current time.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void cleanExpiredMobFiles(FileSystem fs, Configuration conf, TableName tableName,
|
||||
HColumnDescriptor columnDescriptor, CacheConfig cacheConfig, long current)
|
||||
throws IOException {
|
||||
long timeToLive = columnDescriptor.getTimeToLive();
|
||||
if (Integer.MAX_VALUE == timeToLive) {
|
||||
// no need to clean, because the TTL is not set.
|
||||
return;
|
||||
}
|
||||
|
||||
Date expireDate = new Date(current - timeToLive * 1000);
|
||||
expireDate = new Date(expireDate.getYear(), expireDate.getMonth(), expireDate.getDate());
|
||||
LOG.info("MOB HFiles older than " + expireDate.toGMTString() + " will be deleted!");
|
||||
|
||||
FileStatus[] stats = null;
|
||||
Path mobTableDir = FSUtils.getTableDir(getMobHome(conf), tableName);
|
||||
Path path = getMobFamilyPath(conf, tableName, columnDescriptor.getNameAsString());
|
||||
try {
|
||||
stats = fs.listStatus(path);
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.warn("Failed to find the mob file " + path, e);
|
||||
}
|
||||
if (null == stats) {
|
||||
// no file found
|
||||
return;
|
||||
}
|
||||
List<StoreFile> filesToClean = new ArrayList<StoreFile>();
|
||||
int deletedFileCount = 0;
|
||||
for (FileStatus file : stats) {
|
||||
String fileName = file.getPath().getName();
|
||||
try {
|
||||
MobFileName mobFileName = null;
|
||||
if (!HFileLink.isHFileLink(file.getPath())) {
|
||||
mobFileName = MobFileName.create(fileName);
|
||||
} else {
|
||||
HFileLink hfileLink = HFileLink.buildFromHFileLinkPattern(conf, file.getPath());
|
||||
mobFileName = MobFileName.create(hfileLink.getOriginPath().getName());
|
||||
}
|
||||
Date fileDate = parseDate(mobFileName.getDate());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Checking file " + fileName);
|
||||
}
|
||||
if (fileDate.getTime() < expireDate.getTime()) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(fileName + " is an expired file");
|
||||
}
|
||||
filesToClean.add(new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Cannot parse the fileName " + fileName, e);
|
||||
}
|
||||
}
|
||||
if (!filesToClean.isEmpty()) {
|
||||
try {
|
||||
removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(),
|
||||
filesToClean);
|
||||
deletedFileCount = filesToClean.size();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to delete the mob files " + filesToClean, e);
|
||||
}
|
||||
}
|
||||
LOG.info(deletedFileCount + " expired mob files are deleted");
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the root dir of the mob files.
|
||||
* It's {HBASE_DIR}/mobdir.
|
||||
* @param conf The current configuration.
|
||||
* @return the root dir of the mob file.
|
||||
*/
|
||||
public static Path getMobHome(Configuration conf) {
|
||||
Path hbaseDir = new Path(conf.get(HConstants.HBASE_DIR));
|
||||
return new Path(hbaseDir, MobConstants.MOB_DIR_NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the qualified root dir of the mob files.
|
||||
* @param conf The current configuration.
|
||||
* @return The qualified root dir.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Path getQualifiedMobRootDir(Configuration conf) throws IOException {
|
||||
Path hbaseDir = new Path(conf.get(HConstants.HBASE_DIR));
|
||||
Path mobRootDir = new Path(hbaseDir, MobConstants.MOB_DIR_NAME);
|
||||
FileSystem fs = mobRootDir.getFileSystem(conf);
|
||||
return mobRootDir.makeQualified(fs);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the region dir of the mob files.
|
||||
* It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}.
|
||||
* @param conf The current configuration.
|
||||
* @param tableName The current table name.
|
||||
* @return The region dir of the mob files.
|
||||
*/
|
||||
public static Path getMobRegionPath(Configuration conf, TableName tableName) {
|
||||
Path tablePath = FSUtils.getTableDir(getMobHome(conf), tableName);
|
||||
HRegionInfo regionInfo = getMobRegionInfo(tableName);
|
||||
return new Path(tablePath, regionInfo.getEncodedName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the family dir of the mob files.
|
||||
* It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
|
||||
* @param conf The current configuration.
|
||||
* @param tableName The current table name.
|
||||
* @param familyName The current family name.
|
||||
* @return The family dir of the mob files.
|
||||
*/
|
||||
public static Path getMobFamilyPath(Configuration conf, TableName tableName, String familyName) {
|
||||
return new Path(getMobRegionPath(conf, tableName), familyName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the family dir of the mob files.
|
||||
* It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
|
||||
* @param regionPath The path of mob region which is a dummy one.
|
||||
* @param familyName The current family name.
|
||||
* @return The family dir of the mob files.
|
||||
*/
|
||||
public static Path getMobFamilyPath(Path regionPath, String familyName) {
|
||||
return new Path(regionPath, familyName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the HRegionInfo of the mob files.
|
||||
* This is a dummy region. The mob files are not saved in a region in HBase.
|
||||
* This is only used in mob snapshot. It's internally used only.
|
||||
* @param tableName
|
||||
* @return A dummy mob region info.
|
||||
*/
|
||||
public static HRegionInfo getMobRegionInfo(TableName tableName) {
|
||||
HRegionInfo info = new HRegionInfo(tableName, MobConstants.MOB_REGION_NAME_BYTES,
|
||||
HConstants.EMPTY_END_ROW, false, 0);
|
||||
return info;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets whether the current HRegionInfo is a mob one.
|
||||
* @param regionInfo The current HRegionInfo.
|
||||
* @return If true, the current HRegionInfo is a mob one.
|
||||
*/
|
||||
public static boolean isMobRegionInfo(HRegionInfo regionInfo) {
|
||||
return regionInfo == null ? false : getMobRegionInfo(regionInfo.getTable()).getEncodedName()
|
||||
.equals(regionInfo.getEncodedName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets whether the current region name follows the pattern of a mob region name.
|
||||
* @param tableName The current table name.
|
||||
* @param regionName The current region name.
|
||||
* @return True if the current region name follows the pattern of a mob region name.
|
||||
*/
|
||||
public static boolean isMobRegionName(TableName tableName, byte[] regionName) {
|
||||
return Bytes.equals(regionName, getMobRegionInfo(tableName).getRegionName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the working directory of the mob compaction.
|
||||
* @param root The root directory of the mob compaction.
|
||||
* @param jobName The current job name.
|
||||
* @return The directory of the mob compaction for the current job.
|
||||
*/
|
||||
public static Path getCompactionWorkingPath(Path root, String jobName) {
|
||||
return new Path(root, jobName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Archives the mob files.
|
||||
* @param conf The current configuration.
|
||||
* @param fs The current file system.
|
||||
* @param tableName The table name.
|
||||
* @param tableDir The table directory.
|
||||
* @param family The name of the column family.
|
||||
* @param storeFiles The files to be deleted.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName,
|
||||
Path tableDir, byte[] family, Collection<StoreFile> storeFiles) throws IOException {
|
||||
HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family,
|
||||
storeFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a mob reference KeyValue.
|
||||
* The value of the mob reference KeyValue is mobCellValueSize + mobFileName.
|
||||
* @param cell The original Cell.
|
||||
* @param fileName The mob file name where the mob reference KeyValue is written.
|
||||
* @param tableNameTag The tag of the current table name. It's very important in
|
||||
* cloning the snapshot.
|
||||
* @return The mob reference KeyValue.
|
||||
*/
|
||||
public static KeyValue createMobRefKeyValue(Cell cell, byte[] fileName, Tag tableNameTag) {
|
||||
// Append the tags to the KeyValue.
|
||||
// The key is same, the value is the filename of the mob file
|
||||
List<Tag> tags = new ArrayList<Tag>();
|
||||
// Add the ref tag as the 1st one.
|
||||
tags.add(MobConstants.MOB_REF_TAG);
|
||||
// Add the tag of the source table name, this table is where this mob file is flushed
|
||||
// from.
|
||||
// It's very useful in cloning the snapshot. When reading from the cloning table, we need to
|
||||
// find the original mob files by this table name. For details please see cloning
|
||||
// snapshot for mob files.
|
||||
tags.add(tableNameTag);
|
||||
// Add the existing tags.
|
||||
tags.addAll(Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength()));
|
||||
int valueLength = cell.getValueLength();
|
||||
byte[] refValue = Bytes.add(Bytes.toBytes(valueLength), fileName);
|
||||
KeyValue reference = new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
|
||||
cell.getTimestamp(), KeyValue.Type.Put, refValue, 0, refValue.length, tags);
|
||||
reference.setSequenceId(cell.getSequenceId());
|
||||
return reference;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a writer for the mob file in temp directory.
|
||||
* @param conf The current configuration.
|
||||
* @param fs The current file system.
|
||||
* @param family The descriptor of the current column family.
|
||||
* @param date The date string, its format is yyyymmmdd.
|
||||
* @param basePath The basic path for a temp directory.
|
||||
* @param maxKeyCount The key count.
|
||||
* @param compression The compression algorithm.
|
||||
* @param startKey The hex string of the start key.
|
||||
* @param cacheConfig The current cache config.
|
||||
* @param cryptoContext The encryption context.
|
||||
* @return The writer for the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static StoreFile.Writer createWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||
Compression.Algorithm compression, String startKey, CacheConfig cacheConfig,
|
||||
Encryption.Context cryptoContext)
|
||||
throws IOException {
|
||||
MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID().toString()
|
||||
.replaceAll("-", ""));
|
||||
return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
|
||||
cacheConfig, cryptoContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a writer for the ref file in temp directory.
|
||||
* @param conf The current configuration.
|
||||
* @param fs The current file system.
|
||||
* @param family The descriptor of the current column family.
|
||||
* @param basePath The basic path for a temp directory.
|
||||
* @param maxKeyCount The key count.
|
||||
* @param cacheConfig The current cache config.
|
||||
* @param cryptoContext The encryption context.
|
||||
* @return The writer for the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static StoreFile.Writer createRefFileWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, Path basePath, long maxKeyCount, CacheConfig cacheConfig,
|
||||
Encryption.Context cryptoContext)
|
||||
throws IOException {
|
||||
HFileContext hFileContext = new HFileContextBuilder().withIncludesMvcc(true)
|
||||
.withIncludesTags(true).withCompression(family.getCompactionCompression())
|
||||
.withCompressTags(family.isCompressTags()).withChecksumType(HStore.getChecksumType(conf))
|
||||
.withBytesPerCheckSum(HStore.getBytesPerChecksum(conf)).withBlockSize(family.getBlocksize())
|
||||
.withHBaseCheckSum(true).withDataBlockEncoding(family.getDataBlockEncoding())
|
||||
.withEncryptionContext(cryptoContext).withCreateTime(EnvironmentEdgeManager.currentTime())
|
||||
.build();
|
||||
Path tempPath = new Path(basePath, UUID.randomUUID().toString().replaceAll("-", ""));
|
||||
StoreFile.Writer w = new StoreFile.WriterBuilder(conf, cacheConfig, fs).withFilePath(tempPath)
|
||||
.withComparator(CellComparator.COMPARATOR).withBloomType(family.getBloomFilterType())
|
||||
.withMaxKeyCount(maxKeyCount).withFileContext(hFileContext).build();
|
||||
return w;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a writer for the mob file in temp directory.
|
||||
* @param conf The current configuration.
|
||||
* @param fs The current file system.
|
||||
* @param family The descriptor of the current column family.
|
||||
* @param date The date string, its format is yyyymmmdd.
|
||||
* @param basePath The basic path for a temp directory.
|
||||
* @param maxKeyCount The key count.
|
||||
* @param compression The compression algorithm.
|
||||
* @param startKey The start key.
|
||||
* @param cacheConfig The current cache config.
|
||||
* @param cryptoContext The encryption context.
|
||||
* @return The writer for the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static StoreFile.Writer createWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||
Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig,
|
||||
Encryption.Context cryptoContext)
|
||||
throws IOException {
|
||||
MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID().toString()
|
||||
.replaceAll("-", ""));
|
||||
return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
|
||||
cacheConfig, cryptoContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a writer for the del file in temp directory.
|
||||
* @param conf The current configuration.
|
||||
* @param fs The current file system.
|
||||
* @param family The descriptor of the current column family.
|
||||
* @param date The date string, its format is yyyymmmdd.
|
||||
* @param basePath The basic path for a temp directory.
|
||||
* @param maxKeyCount The key count.
|
||||
* @param compression The compression algorithm.
|
||||
* @param startKey The start key.
|
||||
* @param cacheConfig The current cache config.
|
||||
* @param cryptoContext The encryption context.
|
||||
* @return The writer for the del file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static StoreFile.Writer createDelFileWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||
Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig,
|
||||
Encryption.Context cryptoContext)
|
||||
throws IOException {
|
||||
String suffix = UUID
|
||||
.randomUUID().toString().replaceAll("-", "") + "_del";
|
||||
MobFileName mobFileName = MobFileName.create(startKey, date, suffix);
|
||||
return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
|
||||
cacheConfig, cryptoContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a writer for the mob file in temp directory.
|
||||
* @param conf The current configuration.
|
||||
* @param fs The current file system.
|
||||
* @param family The descriptor of the current column family.
|
||||
* @param mobFileName The mob file name.
|
||||
* @param basePath The basic path for a temp directory.
|
||||
* @param maxKeyCount The key count.
|
||||
* @param compression The compression algorithm.
|
||||
* @param cacheConfig The current cache config.
|
||||
* @param cryptoContext The encryption context.
|
||||
* @return The writer for the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
private static StoreFile.Writer createWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount,
|
||||
Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext)
|
||||
throws IOException {
|
||||
HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
|
||||
.withIncludesMvcc(true).withIncludesTags(true)
|
||||
.withCompressTags(family.isCompressTags())
|
||||
.withChecksumType(HStore.getChecksumType(conf))
|
||||
.withBytesPerCheckSum(HStore.getBytesPerChecksum(conf)).withBlockSize(family.getBlocksize())
|
||||
.withHBaseCheckSum(true).withDataBlockEncoding(family.getDataBlockEncoding())
|
||||
.withEncryptionContext(cryptoContext)
|
||||
.withCreateTime(EnvironmentEdgeManager.currentTime()).build();
|
||||
|
||||
StoreFile.Writer w = new StoreFile.WriterBuilder(conf, cacheConfig, fs)
|
||||
.withFilePath(new Path(basePath, mobFileName.getFileName()))
|
||||
.withComparator(CellComparator.COMPARATOR).withBloomType(BloomType.NONE)
|
||||
.withMaxKeyCount(maxKeyCount).withFileContext(hFileContext).build();
|
||||
return w;
|
||||
}
|
||||
|
||||
/**
|
||||
* Commits the mob file.
|
||||
* @param conf The current configuration.
|
||||
* @param fs The current file system.
|
||||
* @param sourceFile The path where the mob file is saved.
|
||||
* @param targetPath The directory path where the source file is renamed to.
|
||||
* @param cacheConfig The current cache config.
|
||||
* @return The target file path the source file is renamed to.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Path commitFile(Configuration conf, FileSystem fs, final Path sourceFile,
|
||||
Path targetPath, CacheConfig cacheConfig) throws IOException {
|
||||
if (sourceFile == null) {
|
||||
return null;
|
||||
}
|
||||
Path dstPath = new Path(targetPath, sourceFile.getName());
|
||||
validateMobFile(conf, fs, sourceFile, cacheConfig);
|
||||
String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
|
||||
LOG.info(msg);
|
||||
Path parent = dstPath.getParent();
|
||||
if (!fs.exists(parent)) {
|
||||
fs.mkdirs(parent);
|
||||
}
|
||||
if (!fs.rename(sourceFile, dstPath)) {
|
||||
throw new IOException("Failed rename of " + sourceFile + " to " + dstPath);
|
||||
}
|
||||
return dstPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates a mob file by opening and closing it.
|
||||
* @param conf The current configuration.
|
||||
* @param fs The current file system.
|
||||
* @param path The path where the mob file is saved.
|
||||
* @param cacheConfig The current cache config.
|
||||
*/
|
||||
private static void validateMobFile(Configuration conf, FileSystem fs, Path path,
|
||||
CacheConfig cacheConfig) throws IOException {
|
||||
StoreFile storeFile = null;
|
||||
try {
|
||||
storeFile = new StoreFile(fs, path, conf, cacheConfig, BloomType.NONE);
|
||||
storeFile.createReader();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to open mob file[" + path + "], keep it in temp directory.", e);
|
||||
throw e;
|
||||
} finally {
|
||||
if (storeFile != null) {
|
||||
storeFile.closeReader(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the current mob ref cell has a valid value.
|
||||
* A mob ref cell has a mob reference tag.
|
||||
* The value of a mob ref cell consists of two parts, real mob value length and mob file name.
|
||||
* The real mob value length takes 4 bytes.
|
||||
* The remaining part is the mob file name.
|
||||
* @param cell The mob ref cell.
|
||||
* @return True if the cell has a valid value.
|
||||
*/
|
||||
public static boolean hasValidMobRefCellValue(Cell cell) {
|
||||
return cell.getValueLength() > Bytes.SIZEOF_INT;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the mob value length from the mob ref cell.
|
||||
* A mob ref cell has a mob reference tag.
|
||||
* The value of a mob ref cell consists of two parts, real mob value length and mob file name.
|
||||
* The real mob value length takes 4 bytes.
|
||||
* The remaining part is the mob file name.
|
||||
* @param cell The mob ref cell.
|
||||
* @return The real mob value length.
|
||||
*/
|
||||
public static int getMobValueLength(Cell cell) {
|
||||
return Bytes.toInt(cell.getValueArray(), cell.getValueOffset(), Bytes.SIZEOF_INT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the mob file name from the mob ref cell.
|
||||
* A mob ref cell has a mob reference tag.
|
||||
* The value of a mob ref cell consists of two parts, real mob value length and mob file name.
|
||||
* The real mob value length takes 4 bytes.
|
||||
* The remaining part is the mob file name.
|
||||
* @param cell The mob ref cell.
|
||||
* @return The mob file name.
|
||||
*/
|
||||
public static String getMobFileName(Cell cell) {
|
||||
return Bytes.toString(cell.getValueArray(), cell.getValueOffset() + Bytes.SIZEOF_INT,
|
||||
cell.getValueLength() - Bytes.SIZEOF_INT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the table name used in the table lock.
|
||||
* The table lock name is a dummy one, it's not a table name. It's tableName + ".mobLock".
|
||||
* @param tn The table name.
|
||||
* @return The table name used in table lock.
|
||||
*/
|
||||
public static TableName getTableLockName(TableName tn) {
|
||||
byte[] tableName = tn.getName();
|
||||
return TableName.valueOf(Bytes.add(tableName, MobConstants.MOB_TABLE_LOCK_SUFFIX));
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs the mob compaction.
|
||||
* @param conf the Configuration
|
||||
* @param fs the file system
|
||||
* @param tableName the table the compact
|
||||
* @param hcd the column descriptor
|
||||
* @param pool the thread pool
|
||||
* @param tableLockManager the tableLock manager
|
||||
* @param allFiles Whether add all mob files into the compaction.
|
||||
*/
|
||||
public static void doMobCompaction(Configuration conf, FileSystem fs, TableName tableName,
|
||||
HColumnDescriptor hcd, ExecutorService pool, TableLockManager tableLockManager,
|
||||
boolean allFiles) throws IOException {
|
||||
String className = conf.get(MobConstants.MOB_COMPACTOR_CLASS_KEY,
|
||||
PartitionedMobCompactor.class.getName());
|
||||
// instantiate the mob compactor.
|
||||
MobCompactor compactor = null;
|
||||
try {
|
||||
compactor = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {
|
||||
Configuration.class, FileSystem.class, TableName.class, HColumnDescriptor.class,
|
||||
ExecutorService.class }, new Object[] { conf, fs, tableName, hcd, pool });
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Unable to load configured mob file compactor '" + className + "'", e);
|
||||
}
|
||||
// compact only for mob-enabled column.
|
||||
// obtain a write table lock before performing compaction to avoid race condition
|
||||
// with major compaction in mob-enabled column.
|
||||
boolean tableLocked = false;
|
||||
TableLock lock = null;
|
||||
try {
|
||||
// the tableLockManager might be null in testing. In that case, it is lock-free.
|
||||
if (tableLockManager != null) {
|
||||
lock = tableLockManager.writeLock(MobUtils.getTableLockName(tableName),
|
||||
"Run MobCompactor");
|
||||
lock.acquire();
|
||||
}
|
||||
tableLocked = true;
|
||||
compactor.compact(allFiles);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to compact the mob files for the column " + hcd.getNameAsString()
|
||||
+ " in the table " + tableName.getNameAsString(), e);
|
||||
} finally {
|
||||
if (lock != null && tableLocked) {
|
||||
try {
|
||||
lock.release();
|
||||
} catch (IOException e) {
|
||||
LOG.error(
|
||||
"Failed to release the write lock for the table " + tableName.getNameAsString(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a thread pool.
|
||||
* @param conf the Configuration
|
||||
* @return A thread pool.
|
||||
*/
|
||||
public static ExecutorService createMobCompactorThreadPool(Configuration conf) {
|
||||
int maxThreads = conf.getInt(MobConstants.MOB_COMPACTION_THREADS_MAX,
|
||||
MobConstants.DEFAULT_MOB_COMPACTION_THREADS_MAX);
|
||||
if (maxThreads == 0) {
|
||||
maxThreads = 1;
|
||||
}
|
||||
final SynchronousQueue<Runnable> queue = new SynchronousQueue<Runnable>();
|
||||
ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS, queue,
|
||||
Threads.newDaemonThreadFactory("MobCompactor"), new RejectedExecutionHandler() {
|
||||
@Override
|
||||
public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
|
||||
try {
|
||||
// waiting for a thread to pick up instead of throwing exceptions.
|
||||
queue.put(r);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RejectedExecutionException(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
|
||||
return pool;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the encyption context.
|
||||
* @param conf The current configuration.
|
||||
* @param family The current column descriptor.
|
||||
* @return The encryption context.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Encryption.Context createEncryptionContext(Configuration conf,
|
||||
HColumnDescriptor family) throws IOException {
|
||||
// TODO the code is repeated, and needs to be unified.
|
||||
Encryption.Context cryptoContext = Encryption.Context.NONE;
|
||||
String cipherName = family.getEncryptionType();
|
||||
if (cipherName != null) {
|
||||
Cipher cipher;
|
||||
Key key;
|
||||
byte[] keyBytes = family.getEncryptionKey();
|
||||
if (keyBytes != null) {
|
||||
// Family provides specific key material
|
||||
String masterKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User
|
||||
.getCurrent().getShortName());
|
||||
try {
|
||||
// First try the master key
|
||||
key = EncryptionUtil.unwrapKey(conf, masterKeyName, keyBytes);
|
||||
} catch (KeyException e) {
|
||||
// If the current master key fails to unwrap, try the alternate, if
|
||||
// one is configured
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Unable to unwrap key with current master key '" + masterKeyName + "'");
|
||||
}
|
||||
String alternateKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY);
|
||||
if (alternateKeyName != null) {
|
||||
try {
|
||||
key = EncryptionUtil.unwrapKey(conf, alternateKeyName, keyBytes);
|
||||
} catch (KeyException ex) {
|
||||
throw new IOException(ex);
|
||||
}
|
||||
} else {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
// Use the algorithm the key wants
|
||||
cipher = Encryption.getCipher(conf, key.getAlgorithm());
|
||||
if (cipher == null) {
|
||||
throw new RuntimeException("Cipher '" + key.getAlgorithm() + "' is not available");
|
||||
}
|
||||
// Fail if misconfigured
|
||||
// We use the encryption type specified in the column schema as a sanity check on
|
||||
// what the wrapped key is telling us
|
||||
if (!cipher.getName().equalsIgnoreCase(cipherName)) {
|
||||
throw new RuntimeException("Encryption for family '" + family.getNameAsString()
|
||||
+ "' configured with type '" + cipherName + "' but key specifies algorithm '"
|
||||
+ cipher.getName() + "'");
|
||||
}
|
||||
} else {
|
||||
// Family does not provide key material, create a random key
|
||||
cipher = Encryption.getCipher(conf, cipherName);
|
||||
if (cipher == null) {
|
||||
throw new RuntimeException("Cipher '" + cipherName + "' is not available");
|
||||
}
|
||||
key = cipher.getRandomKey();
|
||||
}
|
||||
cryptoContext = Encryption.newContext(conf);
|
||||
cryptoContext.setCipher(cipher);
|
||||
cryptoContext.setKey(key);
|
||||
}
|
||||
return cryptoContext;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether this table has mob-enabled columns.
|
||||
* @param htd The current table descriptor.
|
||||
* @return Whether this table has mob-enabled columns.
|
||||
*/
|
||||
public static boolean hasMobColumns(HTableDescriptor htd) {
|
||||
HColumnDescriptor[] hcds = htd.getColumnFamilies();
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
if (hcd.isMobEnabled()) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether return null value when the mob file is missing or corrupt.
|
||||
* The information is set in the attribute "empty.value.on.mobcell.miss" of scan.
|
||||
* @param scan The current scan.
|
||||
* @return True if the readEmptyValueOnMobCellMiss is enabled.
|
||||
*/
|
||||
public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) {
|
||||
byte[] readEmptyValueOnMobCellMiss =
|
||||
scan.getAttribute(MobConstants.EMPTY_VALUE_ON_MOBCELL_MISS);
|
||||
try {
|
||||
return readEmptyValueOnMobCellMiss != null && Bytes.toBoolean(readEmptyValueOnMobCellMiss);
|
||||
} catch (IllegalArgumentException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Archives mob store files
|
||||
* @param conf The current configuration.
|
||||
* @param fs The current file system.
|
||||
* @param mobRegionInfo The mob family region info.
|
||||
* @param mobFamilyDir The mob family directory.
|
||||
* @param family The name of the column family.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void archiveMobStoreFiles(Configuration conf, FileSystem fs,
|
||||
HRegionInfo mobRegionInfo, Path mobFamilyDir, byte[] family) throws IOException {
|
||||
// disable the block cache.
|
||||
Configuration copyOfConf = HBaseConfiguration.create(conf);
|
||||
copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
|
||||
CacheConfig cacheConfig = new CacheConfig(copyOfConf);
|
||||
FileStatus[] fileStatus = FSUtils.listStatus(fs, mobFamilyDir);
|
||||
List<StoreFile> storeFileList = new ArrayList<StoreFile>();
|
||||
for (FileStatus file : fileStatus) {
|
||||
storeFileList.add(new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE));
|
||||
}
|
||||
HFileArchiver.archiveStoreFiles(conf, fs, mobRegionInfo, mobFamilyDir, family, storeFileList);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,64 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.mob.compactions;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* The compaction request for mob files.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class MobCompactionRequest {
|
||||
|
||||
protected long selectionTime;
|
||||
protected CompactionType type = CompactionType.PART_FILES;
|
||||
|
||||
public void setCompactionType(CompactionType type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the selection time.
|
||||
* @return The selection time.
|
||||
*/
|
||||
public long getSelectionTime() {
|
||||
return this.selectionTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the compaction type.
|
||||
* @return The compaction type.
|
||||
*/
|
||||
public CompactionType getCompactionType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
protected enum CompactionType {
|
||||
|
||||
/**
|
||||
* Part of mob files are selected.
|
||||
*/
|
||||
PART_FILES,
|
||||
|
||||
/**
|
||||
* All of mob files are selected.
|
||||
*/
|
||||
ALL_FILES;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.compactions;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
/**
|
||||
* A mob compactor to directly compact the mob files.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class MobCompactor {
|
||||
|
||||
protected FileSystem fs;
|
||||
protected Configuration conf;
|
||||
protected TableName tableName;
|
||||
protected HColumnDescriptor column;
|
||||
|
||||
protected Path mobTableDir;
|
||||
protected Path mobFamilyDir;
|
||||
protected ExecutorService pool;
|
||||
|
||||
public MobCompactor(Configuration conf, FileSystem fs, TableName tableName,
|
||||
HColumnDescriptor column, ExecutorService pool) {
|
||||
this.conf = conf;
|
||||
this.fs = fs;
|
||||
this.tableName = tableName;
|
||||
this.column = column;
|
||||
this.pool = pool;
|
||||
mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
|
||||
mobFamilyDir = MobUtils.getMobFamilyPath(conf, tableName, column.getNameAsString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts the mob files for the current column family.
|
||||
* @return The paths of new mob files generated in the compaction.
|
||||
* @throws IOException
|
||||
*/
|
||||
public List<Path> compact() throws IOException {
|
||||
return compact(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts the mob files by compaction type for the current column family.
|
||||
* @param allFiles Whether add all mob files into the compaction.
|
||||
* @return The paths of new mob files generated in the compaction.
|
||||
* @throws IOException
|
||||
*/
|
||||
public List<Path> compact(boolean allFiles) throws IOException {
|
||||
return compact(Arrays.asList(fs.listStatus(mobFamilyDir)), allFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts the candidate mob files.
|
||||
* @param files The candidate mob files.
|
||||
* @param allFiles Whether add all mob files into the compaction.
|
||||
* @return The paths of new mob files generated in the compaction.
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract List<Path> compact(List<FileStatus> files, boolean allFiles)
|
||||
throws IOException;
|
||||
}
|
|
@ -0,0 +1,146 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.compactions;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
/**
|
||||
* An implementation of {@link MobCompactionRequest} that is used in
|
||||
* {@link PartitionedMobCompactor}.
|
||||
* The mob files that have the same start key and date in their names belong to
|
||||
* the same partition.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class PartitionedMobCompactionRequest extends MobCompactionRequest {
|
||||
|
||||
protected Collection<FileStatus> delFiles;
|
||||
protected Collection<CompactionPartition> compactionPartitions;
|
||||
|
||||
public PartitionedMobCompactionRequest(Collection<CompactionPartition> compactionPartitions,
|
||||
Collection<FileStatus> delFiles) {
|
||||
this.selectionTime = EnvironmentEdgeManager.currentTime();
|
||||
this.compactionPartitions = compactionPartitions;
|
||||
this.delFiles = delFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the compaction partitions.
|
||||
* @return The compaction partitions.
|
||||
*/
|
||||
public Collection<CompactionPartition> getCompactionPartitions() {
|
||||
return this.compactionPartitions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the del files.
|
||||
* @return The del files.
|
||||
*/
|
||||
public Collection<FileStatus> getDelFiles() {
|
||||
return this.delFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* The partition in the mob compaction.
|
||||
* The mob files that have the same start key and date in their names belong to
|
||||
* the same partition.
|
||||
*/
|
||||
protected static class CompactionPartition {
|
||||
private List<FileStatus> files = new ArrayList<FileStatus>();
|
||||
private CompactionPartitionId partitionId;
|
||||
|
||||
public CompactionPartition(CompactionPartitionId partitionId) {
|
||||
this.partitionId = partitionId;
|
||||
}
|
||||
|
||||
public CompactionPartitionId getPartitionId() {
|
||||
return this.partitionId;
|
||||
}
|
||||
|
||||
public void addFile(FileStatus file) {
|
||||
files.add(file);
|
||||
}
|
||||
|
||||
public List<FileStatus> listFiles() {
|
||||
return Collections.unmodifiableList(files);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The partition id that consists of start key and date of the mob file name.
|
||||
*/
|
||||
public static class CompactionPartitionId {
|
||||
|
||||
private String startKey;
|
||||
private String date;
|
||||
|
||||
public CompactionPartitionId(String startKey, String date) {
|
||||
if (startKey == null || date == null) {
|
||||
throw new IllegalArgumentException("Neither of start key and date could be null");
|
||||
}
|
||||
this.startKey = startKey;
|
||||
this.date = date;
|
||||
}
|
||||
|
||||
public String getStartKey() {
|
||||
return this.startKey;
|
||||
}
|
||||
|
||||
public String getDate() {
|
||||
return this.date;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = 17;
|
||||
result = 31 * result + startKey.hashCode();
|
||||
result = 31 * result + date.hashCode();
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (!(obj instanceof CompactionPartitionId)) {
|
||||
return false;
|
||||
}
|
||||
CompactionPartitionId another = (CompactionPartitionId) obj;
|
||||
if (!this.startKey.equals(another.startKey)) {
|
||||
return false;
|
||||
}
|
||||
if (!this.date.equals(another.date)) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringBuilder(startKey).append(date).toString();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,679 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.compactions;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.io.crypto.Encryption;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.mob.MobFileName;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType;
|
||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
|
||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanInfo;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreScanner;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
/**
|
||||
* An implementation of {@link MobCompactor} that compacts the mob files in partitions.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class PartitionedMobCompactor extends MobCompactor {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(PartitionedMobCompactor.class);
|
||||
protected long mergeableSize;
|
||||
protected int delFileMaxCount;
|
||||
/** The number of files compacted in a batch */
|
||||
protected int compactionBatchSize;
|
||||
protected int compactionKVMax;
|
||||
|
||||
private Path tempPath;
|
||||
private Path bulkloadPath;
|
||||
private CacheConfig compactionCacheConfig;
|
||||
private Tag tableNameTag;
|
||||
private Encryption.Context cryptoContext = Encryption.Context.NONE;
|
||||
|
||||
public PartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName,
|
||||
HColumnDescriptor column, ExecutorService pool) throws IOException {
|
||||
super(conf, fs, tableName, column, pool);
|
||||
mergeableSize = conf.getLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
|
||||
MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);
|
||||
delFileMaxCount = conf.getInt(MobConstants.MOB_DELFILE_MAX_COUNT,
|
||||
MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
|
||||
// default is 100
|
||||
compactionBatchSize = conf.getInt(MobConstants.MOB_COMPACTION_BATCH_SIZE,
|
||||
MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE);
|
||||
tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME);
|
||||
bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path(
|
||||
tableName.getNamespaceAsString(), tableName.getQualifierAsString())));
|
||||
compactionKVMax = this.conf.getInt(HConstants.COMPACTION_KV_MAX,
|
||||
HConstants.COMPACTION_KV_MAX_DEFAULT);
|
||||
Configuration copyOfConf = new Configuration(conf);
|
||||
copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
|
||||
compactionCacheConfig = new CacheConfig(copyOfConf);
|
||||
tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName());
|
||||
cryptoContext = MobUtils.createEncryptionContext(copyOfConf, column);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Path> compact(List<FileStatus> files, boolean allFiles) throws IOException {
|
||||
if (files == null || files.isEmpty()) {
|
||||
LOG.info("No candidate mob files");
|
||||
return null;
|
||||
}
|
||||
LOG.info("is allFiles: " + allFiles);
|
||||
// find the files to compact.
|
||||
PartitionedMobCompactionRequest request = select(files, allFiles);
|
||||
// compact the files.
|
||||
return performCompaction(request);
|
||||
}
|
||||
|
||||
/**
|
||||
* Selects the compacted mob/del files.
|
||||
* Iterates the candidates to find out all the del files and small mob files.
|
||||
* @param candidates All the candidates.
|
||||
* @param allFiles Whether add all mob files into the compaction.
|
||||
* @return A compaction request.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected PartitionedMobCompactionRequest select(List<FileStatus> candidates,
|
||||
boolean allFiles) throws IOException {
|
||||
Collection<FileStatus> allDelFiles = new ArrayList<FileStatus>();
|
||||
Map<CompactionPartitionId, CompactionPartition> filesToCompact =
|
||||
new HashMap<CompactionPartitionId, CompactionPartition>();
|
||||
int selectedFileCount = 0;
|
||||
int irrelevantFileCount = 0;
|
||||
for (FileStatus file : candidates) {
|
||||
if (!file.isFile()) {
|
||||
irrelevantFileCount++;
|
||||
continue;
|
||||
}
|
||||
// group the del files and small files.
|
||||
FileStatus linkedFile = file;
|
||||
if (HFileLink.isHFileLink(file.getPath())) {
|
||||
HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath());
|
||||
linkedFile = getLinkedFileStatus(link);
|
||||
if (linkedFile == null) {
|
||||
// If the linked file cannot be found, regard it as an irrelevantFileCount file
|
||||
irrelevantFileCount++;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
if (StoreFileInfo.isDelFile(linkedFile.getPath())) {
|
||||
allDelFiles.add(file);
|
||||
} else if (allFiles || linkedFile.getLen() < mergeableSize) {
|
||||
// add all files if allFiles is true,
|
||||
// otherwise add the small files to the merge pool
|
||||
MobFileName fileName = MobFileName.create(linkedFile.getPath().getName());
|
||||
CompactionPartitionId id = new CompactionPartitionId(fileName.getStartKey(),
|
||||
fileName.getDate());
|
||||
CompactionPartition compactionPartition = filesToCompact.get(id);
|
||||
if (compactionPartition == null) {
|
||||
compactionPartition = new CompactionPartition(id);
|
||||
compactionPartition.addFile(file);
|
||||
filesToCompact.put(id, compactionPartition);
|
||||
} else {
|
||||
compactionPartition.addFile(file);
|
||||
}
|
||||
selectedFileCount++;
|
||||
}
|
||||
}
|
||||
PartitionedMobCompactionRequest request = new PartitionedMobCompactionRequest(
|
||||
filesToCompact.values(), allDelFiles);
|
||||
if (candidates.size() == (allDelFiles.size() + selectedFileCount + irrelevantFileCount)) {
|
||||
// all the files are selected
|
||||
request.setCompactionType(CompactionType.ALL_FILES);
|
||||
}
|
||||
LOG.info("The compaction type is " + request.getCompactionType() + ", the request has "
|
||||
+ allDelFiles.size() + " del files, " + selectedFileCount + " selected files, and "
|
||||
+ irrelevantFileCount + " irrelevant files");
|
||||
return request;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs the compaction on the selected files.
|
||||
* <ol>
|
||||
* <li>Compacts the del files.</li>
|
||||
* <li>Compacts the selected small mob files and all the del files.</li>
|
||||
* <li>If all the candidates are selected, delete the del files.</li>
|
||||
* </ol>
|
||||
* @param request The compaction request.
|
||||
* @return The paths of new mob files generated in the compaction.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected List<Path> performCompaction(PartitionedMobCompactionRequest request)
|
||||
throws IOException {
|
||||
// merge the del files
|
||||
List<Path> delFilePaths = new ArrayList<Path>();
|
||||
for (FileStatus delFile : request.delFiles) {
|
||||
delFilePaths.add(delFile.getPath());
|
||||
}
|
||||
List<Path> newDelPaths = compactDelFiles(request, delFilePaths);
|
||||
List<StoreFile> newDelFiles = new ArrayList<StoreFile>();
|
||||
List<Path> paths = null;
|
||||
try {
|
||||
for (Path newDelPath : newDelPaths) {
|
||||
StoreFile sf = new StoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE);
|
||||
// pre-create reader of a del file to avoid race condition when opening the reader in each
|
||||
// partition.
|
||||
sf.createReader();
|
||||
newDelFiles.add(sf);
|
||||
}
|
||||
LOG.info("After merging, there are " + newDelFiles.size() + " del files");
|
||||
// compact the mob files by partitions.
|
||||
paths = compactMobFiles(request, newDelFiles);
|
||||
LOG.info("After compaction, there are " + paths.size() + " mob files");
|
||||
} finally {
|
||||
closeStoreFileReaders(newDelFiles);
|
||||
}
|
||||
// archive the del files if all the mob files are selected.
|
||||
if (request.type == CompactionType.ALL_FILES && !newDelPaths.isEmpty()) {
|
||||
LOG.info("After a mob compaction with all files selected, archiving the del files "
|
||||
+ newDelPaths);
|
||||
try {
|
||||
MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), newDelFiles);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to archive the del files " + newDelPaths, e);
|
||||
}
|
||||
}
|
||||
return paths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts the selected small mob files and all the del files.
|
||||
* @param request The compaction request.
|
||||
* @param delFiles The del files.
|
||||
* @return The paths of new mob files after compactions.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected List<Path> compactMobFiles(final PartitionedMobCompactionRequest request,
|
||||
final List<StoreFile> delFiles) throws IOException {
|
||||
Collection<CompactionPartition> partitions = request.compactionPartitions;
|
||||
if (partitions == null || partitions.isEmpty()) {
|
||||
LOG.info("No partitions of mob files");
|
||||
return Collections.emptyList();
|
||||
}
|
||||
List<Path> paths = new ArrayList<Path>();
|
||||
Connection c = ConnectionFactory.createConnection(conf);
|
||||
final Table table = c.getTable(tableName);
|
||||
try {
|
||||
Map<CompactionPartitionId, Future<List<Path>>> results =
|
||||
new HashMap<CompactionPartitionId, Future<List<Path>>>();
|
||||
// compact the mob files by partitions in parallel.
|
||||
for (final CompactionPartition partition : partitions) {
|
||||
results.put(partition.getPartitionId(), pool.submit(new Callable<List<Path>>() {
|
||||
@Override
|
||||
public List<Path> call() throws Exception {
|
||||
LOG.info("Compacting mob files for partition " + partition.getPartitionId());
|
||||
return compactMobFilePartition(request, partition, delFiles, table);
|
||||
}
|
||||
}));
|
||||
}
|
||||
// compact the partitions in parallel.
|
||||
List<CompactionPartitionId> failedPartitions = new ArrayList<CompactionPartitionId>();
|
||||
for (Entry<CompactionPartitionId, Future<List<Path>>> result : results.entrySet()) {
|
||||
try {
|
||||
paths.addAll(result.getValue().get());
|
||||
} catch (Exception e) {
|
||||
// just log the error
|
||||
LOG.error("Failed to compact the partition " + result.getKey(), e);
|
||||
failedPartitions.add(result.getKey());
|
||||
}
|
||||
}
|
||||
if (!failedPartitions.isEmpty()) {
|
||||
// if any partition fails in the compaction, directly throw an exception.
|
||||
throw new IOException("Failed to compact the partitions " + failedPartitions);
|
||||
}
|
||||
} finally {
|
||||
try {
|
||||
table.close();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to close the HTable", e);
|
||||
}
|
||||
}
|
||||
return paths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts a partition of selected small mob files and all the del files.
|
||||
* @param request The compaction request.
|
||||
* @param partition A compaction partition.
|
||||
* @param delFiles The del files.
|
||||
* @param table The current table.
|
||||
* @return The paths of new mob files after compactions.
|
||||
* @throws IOException
|
||||
*/
|
||||
private List<Path> compactMobFilePartition(PartitionedMobCompactionRequest request,
|
||||
CompactionPartition partition, List<StoreFile> delFiles, Table table) throws IOException {
|
||||
List<Path> newFiles = new ArrayList<Path>();
|
||||
List<FileStatus> files = partition.listFiles();
|
||||
int offset = 0;
|
||||
Path bulkloadPathOfPartition = new Path(bulkloadPath, partition.getPartitionId().toString());
|
||||
Path bulkloadColumnPath = new Path(bulkloadPathOfPartition, column.getNameAsString());
|
||||
while (offset < files.size()) {
|
||||
int batch = compactionBatchSize;
|
||||
if (files.size() - offset < compactionBatchSize) {
|
||||
batch = files.size() - offset;
|
||||
}
|
||||
if (batch == 1 && delFiles.isEmpty()) {
|
||||
// only one file left and no del files, do not compact it,
|
||||
// and directly add it to the new files.
|
||||
newFiles.add(files.get(offset).getPath());
|
||||
offset++;
|
||||
continue;
|
||||
}
|
||||
// clean the bulkload directory to avoid loading old files.
|
||||
fs.delete(bulkloadPathOfPartition, true);
|
||||
// add the selected mob files and del files into filesToCompact
|
||||
List<StoreFile> filesToCompact = new ArrayList<StoreFile>();
|
||||
for (int i = offset; i < batch + offset; i++) {
|
||||
StoreFile sf = new StoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
|
||||
BloomType.NONE);
|
||||
filesToCompact.add(sf);
|
||||
}
|
||||
filesToCompact.addAll(delFiles);
|
||||
// compact the mob files in a batch.
|
||||
compactMobFilesInBatch(request, partition, table, filesToCompact, batch,
|
||||
bulkloadPathOfPartition, bulkloadColumnPath, newFiles);
|
||||
// move to the next batch.
|
||||
offset += batch;
|
||||
}
|
||||
LOG.info("Compaction is finished. The number of mob files is changed from " + files.size()
|
||||
+ " to " + newFiles.size());
|
||||
return newFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the readers of store files.
|
||||
* @param storeFiles The store files to be closed.
|
||||
*/
|
||||
private void closeStoreFileReaders(List<StoreFile> storeFiles) {
|
||||
for (StoreFile storeFile : storeFiles) {
|
||||
try {
|
||||
storeFile.closeReader(true);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts a partition of selected small mob files and all the del files in a batch.
|
||||
* @param request The compaction request.
|
||||
* @param partition A compaction partition.
|
||||
* @param table The current table.
|
||||
* @param filesToCompact The files to be compacted.
|
||||
* @param batch The number of mob files to be compacted in a batch.
|
||||
* @param bulkloadPathOfPartition The directory where the bulkload column of the current
|
||||
* partition is saved.
|
||||
* @param bulkloadColumnPath The directory where the bulkload files of current partition
|
||||
* are saved.
|
||||
* @param newFiles The paths of new mob files after compactions.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void compactMobFilesInBatch(PartitionedMobCompactionRequest request,
|
||||
CompactionPartition partition, Table table, List<StoreFile> filesToCompact, int batch,
|
||||
Path bulkloadPathOfPartition, Path bulkloadColumnPath, List<Path> newFiles)
|
||||
throws IOException {
|
||||
// open scanner to the selected mob files and del files.
|
||||
StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES);
|
||||
// the mob files to be compacted, not include the del files.
|
||||
List<StoreFile> mobFilesToCompact = filesToCompact.subList(0, batch);
|
||||
// Pair(maxSeqId, cellsCount)
|
||||
Pair<Long, Long> fileInfo = getFileInfo(mobFilesToCompact);
|
||||
// open writers for the mob files and new ref store files.
|
||||
Writer writer = null;
|
||||
Writer refFileWriter = null;
|
||||
Path filePath = null;
|
||||
Path refFilePath = null;
|
||||
long mobCells = 0;
|
||||
try {
|
||||
writer = MobUtils.createWriter(conf, fs, column, partition.getPartitionId().getDate(),
|
||||
tempPath, Long.MAX_VALUE, column.getCompactionCompression(), partition.getPartitionId()
|
||||
.getStartKey(), compactionCacheConfig, cryptoContext);
|
||||
filePath = writer.getPath();
|
||||
byte[] fileName = Bytes.toBytes(filePath.getName());
|
||||
// create a temp file and open a writer for it in the bulkloadPath
|
||||
refFileWriter = MobUtils.createRefFileWriter(conf, fs, column, bulkloadColumnPath, fileInfo
|
||||
.getSecond().longValue(), compactionCacheConfig, cryptoContext);
|
||||
refFilePath = refFileWriter.getPath();
|
||||
List<Cell> cells = new ArrayList<Cell>();
|
||||
boolean hasMore = false;
|
||||
ScannerContext scannerContext =
|
||||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
||||
do {
|
||||
hasMore = scanner.next(cells, scannerContext);
|
||||
for (Cell cell : cells) {
|
||||
// write the mob cell to the mob file.
|
||||
writer.append(cell);
|
||||
// write the new reference cell to the store file.
|
||||
KeyValue reference = MobUtils.createMobRefKeyValue(cell, fileName, tableNameTag);
|
||||
refFileWriter.append(reference);
|
||||
mobCells++;
|
||||
}
|
||||
cells.clear();
|
||||
} while (hasMore);
|
||||
} finally {
|
||||
// close the scanner.
|
||||
scanner.close();
|
||||
// append metadata to the mob file, and close the mob file writer.
|
||||
closeMobFileWriter(writer, fileInfo.getFirst(), mobCells);
|
||||
// append metadata and bulkload info to the ref mob file, and close the writer.
|
||||
closeRefFileWriter(refFileWriter, fileInfo.getFirst(), request.selectionTime);
|
||||
}
|
||||
if (mobCells > 0) {
|
||||
// commit mob file
|
||||
MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
|
||||
// bulkload the ref file
|
||||
bulkloadRefFile(table, bulkloadPathOfPartition, filePath.getName());
|
||||
newFiles.add(new Path(mobFamilyDir, filePath.getName()));
|
||||
} else {
|
||||
// remove the new files
|
||||
// the mob file is empty, delete it instead of committing.
|
||||
deletePath(filePath);
|
||||
// the ref file is empty, delete it instead of committing.
|
||||
deletePath(refFilePath);
|
||||
}
|
||||
// archive the old mob files, do not archive the del files.
|
||||
try {
|
||||
closeStoreFileReaders(mobFilesToCompact);
|
||||
MobUtils
|
||||
.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), mobFilesToCompact);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to archive the files " + mobFilesToCompact, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts the del files in batches which avoids opening too many files.
|
||||
* @param request The compaction request.
|
||||
* @param delFilePaths
|
||||
* @return The paths of new del files after merging or the original files if no merging
|
||||
* is necessary.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected List<Path> compactDelFiles(PartitionedMobCompactionRequest request,
|
||||
List<Path> delFilePaths) throws IOException {
|
||||
if (delFilePaths.size() <= delFileMaxCount) {
|
||||
return delFilePaths;
|
||||
}
|
||||
// when there are more del files than the number that is allowed, merge it firstly.
|
||||
int offset = 0;
|
||||
List<Path> paths = new ArrayList<Path>();
|
||||
while (offset < delFilePaths.size()) {
|
||||
// get the batch
|
||||
int batch = compactionBatchSize;
|
||||
if (delFilePaths.size() - offset < compactionBatchSize) {
|
||||
batch = delFilePaths.size() - offset;
|
||||
}
|
||||
List<StoreFile> batchedDelFiles = new ArrayList<StoreFile>();
|
||||
if (batch == 1) {
|
||||
// only one file left, do not compact it, directly add it to the new files.
|
||||
paths.add(delFilePaths.get(offset));
|
||||
offset++;
|
||||
continue;
|
||||
}
|
||||
for (int i = offset; i < batch + offset; i++) {
|
||||
batchedDelFiles.add(new StoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig,
|
||||
BloomType.NONE));
|
||||
}
|
||||
// compact the del files in a batch.
|
||||
paths.add(compactDelFilesInBatch(request, batchedDelFiles));
|
||||
// move to the next batch.
|
||||
offset += batch;
|
||||
}
|
||||
return compactDelFiles(request, paths);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compacts the del file in a batch.
|
||||
* @param request The compaction request.
|
||||
* @param delFiles The del files.
|
||||
* @return The path of new del file after merging.
|
||||
* @throws IOException
|
||||
*/
|
||||
private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request,
|
||||
List<StoreFile> delFiles) throws IOException {
|
||||
// create a scanner for the del files.
|
||||
StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES);
|
||||
Writer writer = null;
|
||||
Path filePath = null;
|
||||
try {
|
||||
writer = MobUtils.createDelFileWriter(conf, fs, column,
|
||||
MobUtils.formatDate(new Date(request.selectionTime)), tempPath, Long.MAX_VALUE,
|
||||
column.getCompactionCompression(), HConstants.EMPTY_START_ROW, compactionCacheConfig,
|
||||
cryptoContext);
|
||||
filePath = writer.getPath();
|
||||
List<Cell> cells = new ArrayList<Cell>();
|
||||
boolean hasMore = false;
|
||||
ScannerContext scannerContext =
|
||||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
||||
do {
|
||||
hasMore = scanner.next(cells, scannerContext);
|
||||
for (Cell cell : cells) {
|
||||
writer.append(cell);
|
||||
}
|
||||
cells.clear();
|
||||
} while (hasMore);
|
||||
} finally {
|
||||
scanner.close();
|
||||
if (writer != null) {
|
||||
try {
|
||||
writer.close();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to close the writer of the file " + filePath, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
// commit the new del file
|
||||
Path path = MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
|
||||
// archive the old del files
|
||||
try {
|
||||
MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delFiles);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to archive the old del files " + delFiles, e);
|
||||
}
|
||||
return path;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a store scanner.
|
||||
* @param filesToCompact The files to be compacted.
|
||||
* @param scanType The scan type.
|
||||
* @return The store scanner.
|
||||
* @throws IOException
|
||||
*/
|
||||
private StoreScanner createScanner(List<StoreFile> filesToCompact, ScanType scanType)
|
||||
throws IOException {
|
||||
List scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, true, false,
|
||||
null, HConstants.LATEST_TIMESTAMP);
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(column.getMaxVersions());
|
||||
long ttl = HStore.determineTTLFromFamily(column);
|
||||
ScanInfo scanInfo = new ScanInfo(column, ttl, 0, CellComparator.COMPARATOR);
|
||||
StoreScanner scanner = new StoreScanner(scan, scanInfo, scanType, null, scanners, 0L,
|
||||
HConstants.LATEST_TIMESTAMP);
|
||||
return scanner;
|
||||
}
|
||||
|
||||
/**
|
||||
* Bulkloads the current file.
|
||||
* @param table The current table.
|
||||
* @param bulkloadDirectory The path of bulkload directory.
|
||||
* @param fileName The current file name.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void bulkloadRefFile(Table table, Path bulkloadDirectory, String fileName)
|
||||
throws IOException {
|
||||
// bulkload the ref file
|
||||
try {
|
||||
LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
|
||||
bulkload.doBulkLoad(bulkloadDirectory, (HTable)table);
|
||||
} catch (Exception e) {
|
||||
// delete the committed mob file
|
||||
deletePath(new Path(mobFamilyDir, fileName));
|
||||
throw new IOException(e);
|
||||
} finally {
|
||||
// delete the bulkload files in bulkloadPath
|
||||
deletePath(bulkloadDirectory);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the mob file writer.
|
||||
* @param writer The mob file writer.
|
||||
* @param maxSeqId Maximum sequence id.
|
||||
* @param mobCellsCount The number of mob cells.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void closeMobFileWriter(Writer writer, long maxSeqId, long mobCellsCount)
|
||||
throws IOException {
|
||||
if (writer != null) {
|
||||
writer.appendMetadata(maxSeqId, false, mobCellsCount);
|
||||
try {
|
||||
writer.close();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to close the writer of the file " + writer.getPath(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the ref file writer.
|
||||
* @param writer The ref file writer.
|
||||
* @param maxSeqId Maximum sequence id.
|
||||
* @param bulkloadTime The timestamp at which the bulk load file is created.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void closeRefFileWriter(Writer writer, long maxSeqId, long bulkloadTime)
|
||||
throws IOException {
|
||||
if (writer != null) {
|
||||
writer.appendMetadata(maxSeqId, false);
|
||||
writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime));
|
||||
writer.appendFileInfo(StoreFile.SKIP_RESET_SEQ_ID, Bytes.toBytes(true));
|
||||
try {
|
||||
writer.close();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to close the writer of the ref file " + writer.getPath(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the max seqId and number of cells of the store files.
|
||||
* @param storeFiles The store files.
|
||||
* @return The pair of the max seqId and number of cells of the store files.
|
||||
* @throws IOException
|
||||
*/
|
||||
private Pair<Long, Long> getFileInfo(List<StoreFile> storeFiles) throws IOException {
|
||||
long maxSeqId = 0;
|
||||
long maxKeyCount = 0;
|
||||
for (StoreFile sf : storeFiles) {
|
||||
// the readers will be closed later after the merge.
|
||||
maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId());
|
||||
byte[] count = sf.createReader().loadFileInfo().get(StoreFile.MOB_CELLS_COUNT);
|
||||
if (count != null) {
|
||||
maxKeyCount += Bytes.toLong(count);
|
||||
}
|
||||
}
|
||||
return new Pair<Long, Long>(Long.valueOf(maxSeqId), Long.valueOf(maxKeyCount));
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes a file.
|
||||
* @param path The path of the file to be deleted.
|
||||
*/
|
||||
private void deletePath(Path path) {
|
||||
try {
|
||||
if (path != null) {
|
||||
fs.delete(path, true);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to delete the file " + path, e);
|
||||
}
|
||||
}
|
||||
|
||||
private FileStatus getLinkedFileStatus(HFileLink link) throws IOException {
|
||||
Path[] locations = link.getLocations();
|
||||
for (Path location : locations) {
|
||||
FileStatus file = getFileStatus(location);
|
||||
if (file != null) {
|
||||
return file;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private FileStatus getFileStatus(Path path) throws IOException {
|
||||
try {
|
||||
if (path != null) {
|
||||
FileStatus file = fs.getFileStatus(path);
|
||||
return file;
|
||||
}
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.warn("The file " + path + " can not be found", e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,185 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.mapreduce;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.io.crypto.Encryption;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
|
||||
import org.apache.hadoop.hbase.mob.mapreduce.SweepJob.SweepCounter;
|
||||
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.MemStore;
|
||||
import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.mapreduce.Reducer.Context;
|
||||
|
||||
/**
|
||||
* The wrapper of a DefaultMemStore.
|
||||
* This wrapper is used in the sweep reducer to buffer and sort the cells written from
|
||||
* the invalid and small mob files.
|
||||
* It's flushed when it's full, the mob data are written to the mob files, and their file names
|
||||
* are written back to store files of HBase.
|
||||
* This memStore is used to sort the cells in mob files.
|
||||
* In a reducer of sweep tool, the mob files are grouped by the same prefix (start key and date),
|
||||
* in each group, the reducer iterates the files and read the cells to a new and bigger mob file.
|
||||
* The cells in the same mob file are ordered, but cells across mob files are not.
|
||||
* So we need this MemStoreWrapper to sort those cells come from different mob files before
|
||||
* flushing them to the disk, when the memStore is big enough it's flushed as a new mob file.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MemStoreWrapper {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(MemStoreWrapper.class);
|
||||
|
||||
private MemStore memstore;
|
||||
private long flushSize;
|
||||
private CompactionPartitionId partitionId;
|
||||
private Context context;
|
||||
private Configuration conf;
|
||||
private BufferedMutator table;
|
||||
private HColumnDescriptor hcd;
|
||||
private Path mobFamilyDir;
|
||||
private FileSystem fs;
|
||||
private CacheConfig cacheConfig;
|
||||
private Encryption.Context cryptoContext = Encryption.Context.NONE;
|
||||
|
||||
public MemStoreWrapper(Context context, FileSystem fs, BufferedMutator table,
|
||||
HColumnDescriptor hcd, MemStore memstore, CacheConfig cacheConfig) throws IOException {
|
||||
this.memstore = memstore;
|
||||
this.context = context;
|
||||
this.fs = fs;
|
||||
this.table = table;
|
||||
this.hcd = hcd;
|
||||
this.conf = context.getConfiguration();
|
||||
this.cacheConfig = cacheConfig;
|
||||
flushSize = this.conf.getLong(MobConstants.MOB_SWEEP_TOOL_COMPACTION_MEMSTORE_FLUSH_SIZE,
|
||||
MobConstants.DEFAULT_MOB_SWEEP_TOOL_COMPACTION_MEMSTORE_FLUSH_SIZE);
|
||||
mobFamilyDir = MobUtils.getMobFamilyPath(conf, table.getName(), hcd.getNameAsString());
|
||||
cryptoContext = MobUtils.createEncryptionContext(conf, hcd);
|
||||
}
|
||||
|
||||
public void setPartitionId(CompactionPartitionId partitionId) {
|
||||
this.partitionId = partitionId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Flushes the memstore if the size is large enough.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void flushMemStoreIfNecessary() throws IOException {
|
||||
if (memstore.heapSize() >= flushSize) {
|
||||
flushMemStore();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Flushes the memstore anyway.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void flushMemStore() throws IOException {
|
||||
MemStoreSnapshot snapshot = memstore.snapshot();
|
||||
internalFlushCache(snapshot);
|
||||
memstore.clearSnapshot(snapshot.getId());
|
||||
}
|
||||
|
||||
/**
|
||||
* Flushes the snapshot of the memstore.
|
||||
* Flushes the mob data to the mob files, and flushes the name of these mob files to HBase.
|
||||
* @param snapshot The snapshot of the memstore.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void internalFlushCache(final MemStoreSnapshot snapshot)
|
||||
throws IOException {
|
||||
if (snapshot.getCellsCount() == 0) {
|
||||
return;
|
||||
}
|
||||
// generate the files into a temp directory.
|
||||
String tempPathString = context.getConfiguration().get(SweepJob.WORKING_FILES_DIR_KEY);
|
||||
StoreFile.Writer mobFileWriter = MobUtils.createWriter(conf, fs, hcd, partitionId.getDate(),
|
||||
new Path(tempPathString), snapshot.getCellsCount(), hcd.getCompactionCompression(),
|
||||
partitionId.getStartKey(), cacheConfig, cryptoContext);
|
||||
|
||||
String relativePath = mobFileWriter.getPath().getName();
|
||||
LOG.info("Create files under a temp directory " + mobFileWriter.getPath().toString());
|
||||
|
||||
byte[] referenceValue = Bytes.toBytes(relativePath);
|
||||
KeyValueScanner scanner = snapshot.getScanner();
|
||||
Cell cell = null;
|
||||
while (null != (cell = scanner.next())) {
|
||||
mobFileWriter.append(cell);
|
||||
}
|
||||
scanner.close();
|
||||
// Write out the log sequence number that corresponds to this output
|
||||
// hfile. The hfile is current up to and including logCacheFlushId.
|
||||
mobFileWriter.appendMetadata(Long.MAX_VALUE, false, snapshot.getCellsCount());
|
||||
mobFileWriter.close();
|
||||
|
||||
MobUtils.commitFile(conf, fs, mobFileWriter.getPath(), mobFamilyDir, cacheConfig);
|
||||
context.getCounter(SweepCounter.FILE_AFTER_MERGE_OR_CLEAN).increment(1);
|
||||
// write reference/fileName back to the store files of HBase.
|
||||
scanner = snapshot.getScanner();
|
||||
scanner.seek(KeyValueUtil.createFirstOnRow(HConstants.EMPTY_START_ROW));
|
||||
cell = null;
|
||||
Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, Bytes.toBytes(this.table.getName()
|
||||
.toString()));
|
||||
long updatedCount = 0;
|
||||
while (null != (cell = scanner.next())) {
|
||||
KeyValue reference = MobUtils.createMobRefKeyValue(cell, referenceValue, tableNameTag);
|
||||
Put put =
|
||||
new Put(reference.getRowArray(), reference.getRowOffset(), reference.getRowLength());
|
||||
put.add(reference);
|
||||
table.mutate(put);
|
||||
updatedCount++;
|
||||
}
|
||||
table.flush();
|
||||
context.getCounter(SweepCounter.RECORDS_UPDATED).increment(updatedCount);
|
||||
scanner.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a KeyValue into the memstore.
|
||||
* @param kv The KeyValue to be added.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void addToMemstore(KeyValue kv) throws IOException {
|
||||
memstore.add(kv);
|
||||
// flush the memstore if it's full.
|
||||
flushMemStoreIfNecessary();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.mapreduce;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.mob.MobFileName;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapreduce.Partitioner;
|
||||
|
||||
/**
|
||||
* The partitioner for the sweep job.
|
||||
* The key is a mob file name. We bucket by date.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobFilePathHashPartitioner extends Partitioner<Text, KeyValue> {
|
||||
|
||||
@Override
|
||||
public int getPartition(Text fileName, KeyValue kv, int numPartitions) {
|
||||
MobFileName mobFileName = MobFileName.create(fileName.toString());
|
||||
String date = mobFileName.getDate();
|
||||
int hash = date.hashCode();
|
||||
return (hash & Integer.MAX_VALUE) % numPartitions;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,603 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.mapreduce;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.PriorityQueue;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.NoTagsKeyValue;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Strings;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.SequenceFile.CompressionType;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.io.serializer.JavaSerialization;
|
||||
import org.apache.hadoop.io.serializer.WritableSerialization;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
|
||||
import org.apache.hadoop.net.DNS;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* The sweep job.
|
||||
* Run map reduce to merge the smaller mob files into bigger ones and cleans the unused ones.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SweepJob {
|
||||
|
||||
private final FileSystem fs;
|
||||
private final Configuration conf;
|
||||
private static final Log LOG = LogFactory.getLog(SweepJob.class);
|
||||
static final String SWEEP_JOB_ID = "hbase.mob.sweep.job.id";
|
||||
static final String SWEEP_JOB_SERVERNAME = "hbase.mob.sweep.job.servername";
|
||||
static final String SWEEP_JOB_TABLE_NODE = "hbase.mob.sweep.job.table.node";
|
||||
static final String WORKING_DIR_KEY = "hbase.mob.sweep.job.dir";
|
||||
static final String WORKING_ALLNAMES_FILE_KEY = "hbase.mob.sweep.job.all.file";
|
||||
static final String WORKING_VISITED_DIR_KEY = "hbase.mob.sweep.job.visited.dir";
|
||||
static final String WORKING_ALLNAMES_DIR = "all";
|
||||
static final String WORKING_VISITED_DIR = "visited";
|
||||
public static final String WORKING_FILES_DIR_KEY = "mob.sweep.job.files.dir";
|
||||
//the MOB_SWEEP_JOB_DELAY is ONE_DAY by default. Its value is only changed when testing.
|
||||
public static final String MOB_SWEEP_JOB_DELAY = "hbase.mob.sweep.job.delay";
|
||||
protected static long ONE_DAY = 24 * 60 * 60 * 1000;
|
||||
private long compactionStartTime = EnvironmentEdgeManager.currentTime();
|
||||
public final static String CREDENTIALS_LOCATION = "credentials_location";
|
||||
private CacheConfig cacheConfig;
|
||||
static final int SCAN_CACHING = 10000;
|
||||
private TableLockManager tableLockManager;
|
||||
|
||||
public SweepJob(Configuration conf, FileSystem fs) {
|
||||
this.conf = conf;
|
||||
this.fs = fs;
|
||||
// disable the block cache.
|
||||
Configuration copyOfConf = new Configuration(conf);
|
||||
copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
|
||||
cacheConfig = new CacheConfig(copyOfConf);
|
||||
}
|
||||
|
||||
static ServerName getCurrentServerName(Configuration conf) throws IOException {
|
||||
String hostname = conf.get(
|
||||
"hbase.regionserver.ipc.address",
|
||||
Strings.domainNamePointerToHostName(DNS.getDefaultHost(
|
||||
conf.get("hbase.regionserver.dns.interface", "default"),
|
||||
conf.get("hbase.regionserver.dns.nameserver", "default"))));
|
||||
int port = conf.getInt(HConstants.REGIONSERVER_PORT, HConstants.DEFAULT_REGIONSERVER_PORT);
|
||||
// Creation of a HSA will force a resolve.
|
||||
InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
|
||||
if (initialIsa.getAddress() == null) {
|
||||
throw new IllegalArgumentException("Failed resolve of " + initialIsa);
|
||||
}
|
||||
return ServerName.valueOf(initialIsa.getHostName(), initialIsa.getPort(),
|
||||
EnvironmentEdgeManager.currentTime());
|
||||
}
|
||||
|
||||
/**
|
||||
* Runs MapReduce to do the sweeping on the mob files.
|
||||
* There's a MobReferenceOnlyFilter so that the mappers only get the cells that have mob
|
||||
* references from 'normal' regions' rows.
|
||||
* The running of the sweep tool on the same column family are mutually exclusive.
|
||||
* The HBase major compaction and running of the sweep tool on the same column family
|
||||
* are mutually exclusive.
|
||||
* The synchronization is done by the Zookeeper.
|
||||
* So in the beginning of the running, we need to make sure only this sweep tool is the only one
|
||||
* that is currently running in this column family, and in this column family there're no major
|
||||
* compaction in progress.
|
||||
* @param tn The current table name.
|
||||
* @param family The descriptor of the current column family.
|
||||
* @return 0 upon success, 3 if bailing out because another compaction is currently happening,
|
||||
* or 4 the mr job was unsuccessful
|
||||
*
|
||||
* @throws IOException
|
||||
* @throws ClassNotFoundException
|
||||
* @throws InterruptedException
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public int sweep(TableName tn, HColumnDescriptor family) throws IOException,
|
||||
ClassNotFoundException, InterruptedException, KeeperException {
|
||||
Configuration conf = new Configuration(this.conf);
|
||||
// check whether the current user is the same one with the owner of hbase root
|
||||
String currentUserName = UserGroupInformation.getCurrentUser().getShortUserName();
|
||||
FileStatus[] hbaseRootFileStat = fs.listStatus(new Path(conf.get(HConstants.HBASE_DIR)));
|
||||
if (hbaseRootFileStat.length > 0) {
|
||||
String owner = hbaseRootFileStat[0].getOwner();
|
||||
if (!owner.equals(currentUserName)) {
|
||||
String errorMsg = "The current user[" + currentUserName
|
||||
+ "] doesn't have hbase root credentials."
|
||||
+ " Please make sure the user is the root of the target HBase";
|
||||
LOG.error(errorMsg);
|
||||
throw new IOException(errorMsg);
|
||||
}
|
||||
} else {
|
||||
LOG.error("The target HBase doesn't exist");
|
||||
throw new IOException("The target HBase doesn't exist");
|
||||
}
|
||||
String familyName = family.getNameAsString();
|
||||
String id = "SweepJob" + UUID.randomUUID().toString().replace("-", "");
|
||||
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, id, new DummyMobAbortable());
|
||||
try {
|
||||
ServerName serverName = getCurrentServerName(conf);
|
||||
tableLockManager = TableLockManager.createTableLockManager(conf, zkw, serverName);
|
||||
TableName lockName = MobUtils.getTableLockName(tn);
|
||||
TableLock lock = tableLockManager.writeLock(lockName, "Run sweep tool");
|
||||
String tableName = tn.getNameAsString();
|
||||
// Try to obtain the lock. Use this lock to synchronize all the query
|
||||
try {
|
||||
lock.acquire();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Can not lock the table " + tableName
|
||||
+ ". The major compaction in HBase may be in-progress or another sweep job is running."
|
||||
+ " Please re-run the job.");
|
||||
return 3;
|
||||
}
|
||||
Job job = null;
|
||||
try {
|
||||
Scan scan = new Scan();
|
||||
scan.addFamily(family.getName());
|
||||
// Do not retrieve the mob data when scanning
|
||||
scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
|
||||
scan.setAttribute(MobConstants.MOB_SCAN_REF_ONLY, Bytes.toBytes(Boolean.TRUE));
|
||||
scan.setCaching(SCAN_CACHING);
|
||||
scan.setCacheBlocks(false);
|
||||
scan.setMaxVersions(family.getMaxVersions());
|
||||
conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY,
|
||||
JavaSerialization.class.getName() + "," + WritableSerialization.class.getName());
|
||||
conf.set(SWEEP_JOB_ID, id);
|
||||
conf.set(SWEEP_JOB_SERVERNAME, serverName.toString());
|
||||
String tableLockNode = ZKUtil.joinZNode(zkw.tableLockZNode, lockName.getNameAsString());
|
||||
conf.set(SWEEP_JOB_TABLE_NODE, tableLockNode);
|
||||
job = prepareJob(tn, familyName, scan, conf);
|
||||
job.getConfiguration().set(TableInputFormat.SCAN_COLUMN_FAMILY, familyName);
|
||||
// Record the compaction start time.
|
||||
// In the sweep tool, only the mob file whose modification time is older than
|
||||
// (startTime - delay) could be handled by this tool.
|
||||
// The delay is one day. It could be configured as well, but this is only used
|
||||
// in the test.
|
||||
job.getConfiguration().setLong(MobConstants.MOB_SWEEP_TOOL_COMPACTION_START_DATE,
|
||||
compactionStartTime);
|
||||
|
||||
job.setPartitionerClass(MobFilePathHashPartitioner.class);
|
||||
submit(job, tn, familyName);
|
||||
if (job.waitForCompletion(true)) {
|
||||
// Archive the unused mob files.
|
||||
removeUnusedFiles(job, tn, family);
|
||||
} else {
|
||||
System.err.println("Job was not successful");
|
||||
return 4;
|
||||
}
|
||||
} finally {
|
||||
try {
|
||||
cleanup(job, tn, familyName);
|
||||
} finally {
|
||||
try {
|
||||
lock.release();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to release the table lock " + tableName, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
zkw.close();
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Prepares a map reduce job.
|
||||
* @param tn The current table name.
|
||||
* @param familyName The current family name.
|
||||
* @param scan The current scan.
|
||||
* @param conf The current configuration.
|
||||
* @return A map reduce job.
|
||||
* @throws IOException
|
||||
*/
|
||||
private Job prepareJob(TableName tn, String familyName, Scan scan, Configuration conf)
|
||||
throws IOException {
|
||||
Job job = Job.getInstance(conf);
|
||||
job.setJarByClass(SweepMapper.class);
|
||||
TableMapReduceUtil.initTableMapperJob(tn.getNameAsString(), scan,
|
||||
SweepMapper.class, Text.class, Writable.class, job);
|
||||
|
||||
job.setInputFormatClass(TableInputFormat.class);
|
||||
job.setMapOutputKeyClass(Text.class);
|
||||
job.setMapOutputValueClass(NoTagsKeyValue.class);
|
||||
job.setReducerClass(SweepReducer.class);
|
||||
job.setOutputFormatClass(NullOutputFormat.class);
|
||||
String jobName = getCustomJobName(this.getClass().getSimpleName(), tn, familyName);
|
||||
job.setJobName(jobName);
|
||||
if (StringUtils.isNotEmpty(conf.get(CREDENTIALS_LOCATION))) {
|
||||
String fileLoc = conf.get(CREDENTIALS_LOCATION);
|
||||
Credentials cred = Credentials.readTokenStorageFile(new File(fileLoc), conf);
|
||||
job.getCredentials().addAll(cred);
|
||||
}
|
||||
return job;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a customized job name.
|
||||
* It's className-mapperClassName-reducerClassName-tableName-familyName.
|
||||
* @param className The current class name.
|
||||
* @param tableName The current table name.
|
||||
* @param familyName The current family name.
|
||||
* @return The customized job name.
|
||||
*/
|
||||
private static String getCustomJobName(String className, TableName tableName, String familyName) {
|
||||
StringBuilder name = new StringBuilder();
|
||||
name.append(className);
|
||||
name.append('-').append(SweepMapper.class.getSimpleName());
|
||||
name.append('-').append(SweepReducer.class.getSimpleName());
|
||||
name.append('-').append(tableName.getNamespaceAsString());
|
||||
name.append('-').append(tableName.getQualifierAsString());
|
||||
name.append('-').append(familyName);
|
||||
return name.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Submits a job.
|
||||
* @param job The current job.
|
||||
* @param tn The current table name.
|
||||
* @param familyName The current family name.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void submit(Job job, TableName tn, String familyName) throws IOException {
|
||||
// delete the temp directory of the mob files in case the failure in the previous
|
||||
// execution.
|
||||
Path tempDir =
|
||||
new Path(MobUtils.getMobHome(job.getConfiguration()), MobConstants.TEMP_DIR_NAME);
|
||||
Path mobCompactionTempDir =
|
||||
new Path(tempDir, MobConstants.MOB_SWEEP_TOOL_COMPACTION_TEMP_DIR_NAME);
|
||||
Path workingPath = MobUtils.getCompactionWorkingPath(mobCompactionTempDir, job.getJobName());
|
||||
job.getConfiguration().set(WORKING_DIR_KEY, workingPath.toString());
|
||||
// delete the working directory in case it'not deleted by the last running.
|
||||
fs.delete(workingPath, true);
|
||||
// create the working directory.
|
||||
fs.mkdirs(workingPath);
|
||||
// create a sequence file which contains the names of all the existing files.
|
||||
Path workingPathOfFiles = new Path(workingPath, "files");
|
||||
Path workingPathOfNames = new Path(workingPath, "names");
|
||||
job.getConfiguration().set(WORKING_FILES_DIR_KEY, workingPathOfFiles.toString());
|
||||
Path allFileNamesPath = new Path(workingPathOfNames, WORKING_ALLNAMES_DIR);
|
||||
job.getConfiguration().set(WORKING_ALLNAMES_FILE_KEY, allFileNamesPath.toString());
|
||||
Path vistiedFileNamesPath = new Path(workingPathOfNames, WORKING_VISITED_DIR);
|
||||
job.getConfiguration().set(WORKING_VISITED_DIR_KEY, vistiedFileNamesPath.toString());
|
||||
// create a directory where the files contain names of visited mob files are saved.
|
||||
fs.mkdirs(vistiedFileNamesPath);
|
||||
Path mobStorePath = MobUtils.getMobFamilyPath(job.getConfiguration(), tn, familyName);
|
||||
// Find all the files whose creation time are older than one day.
|
||||
// Write those file names to a file.
|
||||
// In each reducer there's a writer, it write the visited file names to a file which is saved
|
||||
// in WORKING_VISITED_DIR.
|
||||
// After the job is finished, compare those files, then find out the unused mob files and
|
||||
// archive them.
|
||||
FileStatus[] files = fs.listStatus(mobStorePath);
|
||||
Set<String> fileNames = new TreeSet<String>();
|
||||
long mobCompactionDelay = job.getConfiguration().getLong(MOB_SWEEP_JOB_DELAY, ONE_DAY);
|
||||
for (FileStatus fileStatus : files) {
|
||||
if (fileStatus.isFile() && !HFileLink.isHFileLink(fileStatus.getPath())) {
|
||||
if (compactionStartTime - fileStatus.getModificationTime() > mobCompactionDelay) {
|
||||
// only record the potentially unused files older than one day.
|
||||
fileNames.add(fileStatus.getPath().getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
FSDataOutputStream fout = null;
|
||||
SequenceFile.Writer writer = null;
|
||||
try {
|
||||
// create a file includes all the existing mob files whose creation time is older than
|
||||
// (now - oneDay)
|
||||
fout = fs.create(allFileNamesPath, true);
|
||||
// write the names to a sequence file
|
||||
writer = SequenceFile.createWriter(job.getConfiguration(), fout, String.class, String.class,
|
||||
CompressionType.NONE, null);
|
||||
for (String fileName : fileNames) {
|
||||
writer.append(fileName, MobConstants.EMPTY_STRING);
|
||||
}
|
||||
writer.hflush();
|
||||
} finally {
|
||||
if (writer != null) {
|
||||
IOUtils.closeStream(writer);
|
||||
}
|
||||
if (fout != null) {
|
||||
IOUtils.closeStream(fout);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the unused mob files.
|
||||
* Compare the file which contains all the existing mob files and the visited files,
|
||||
* find out the unused mob file and archive them.
|
||||
* @param conf The current configuration.
|
||||
* @return The unused mob files.
|
||||
* @throws IOException
|
||||
*/
|
||||
List<String> getUnusedFiles(Configuration conf) throws IOException {
|
||||
// find out the unused files and archive them
|
||||
Path allFileNamesPath = new Path(conf.get(WORKING_ALLNAMES_FILE_KEY));
|
||||
SequenceFile.Reader allNamesReader = null;
|
||||
MergeSortReader visitedNamesReader = null;
|
||||
List<String> toBeArchived = new ArrayList<String>();
|
||||
try {
|
||||
allNamesReader = new SequenceFile.Reader(fs, allFileNamesPath, conf);
|
||||
visitedNamesReader = new MergeSortReader(fs, conf,
|
||||
new Path(conf.get(WORKING_VISITED_DIR_KEY)));
|
||||
String nextAll = (String) allNamesReader.next((String) null);
|
||||
String nextVisited = visitedNamesReader.next();
|
||||
do {
|
||||
if (nextAll != null) {
|
||||
if (nextVisited != null) {
|
||||
int compare = nextAll.compareTo(nextVisited);
|
||||
if (compare < 0) {
|
||||
toBeArchived.add(nextAll);
|
||||
nextAll = (String) allNamesReader.next((String) null);
|
||||
} else if (compare > 0) {
|
||||
nextVisited = visitedNamesReader.next();
|
||||
} else {
|
||||
nextAll = (String) allNamesReader.next((String) null);
|
||||
nextVisited = visitedNamesReader.next();
|
||||
}
|
||||
} else {
|
||||
toBeArchived.add(nextAll);
|
||||
nextAll = (String) allNamesReader.next((String) null);
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
} while (nextAll != null || nextVisited != null);
|
||||
} finally {
|
||||
if (allNamesReader != null) {
|
||||
IOUtils.closeStream(allNamesReader);
|
||||
}
|
||||
if (visitedNamesReader != null) {
|
||||
visitedNamesReader.close();
|
||||
}
|
||||
}
|
||||
return toBeArchived;
|
||||
}
|
||||
|
||||
/**
|
||||
* Archives unused mob files.
|
||||
* @param job The current job.
|
||||
* @param tn The current table name.
|
||||
* @param hcd The descriptor of the current column family.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void removeUnusedFiles(Job job, TableName tn, HColumnDescriptor hcd) throws IOException {
|
||||
// find out the unused files and archive them
|
||||
List<StoreFile> storeFiles = new ArrayList<StoreFile>();
|
||||
List<String> toBeArchived = getUnusedFiles(job.getConfiguration());
|
||||
// archive them
|
||||
Path mobStorePath = MobUtils
|
||||
.getMobFamilyPath(job.getConfiguration(), tn, hcd.getNameAsString());
|
||||
for (String archiveFileName : toBeArchived) {
|
||||
Path path = new Path(mobStorePath, archiveFileName);
|
||||
storeFiles.add(new StoreFile(fs, path, job.getConfiguration(), cacheConfig, BloomType.NONE));
|
||||
}
|
||||
if (!storeFiles.isEmpty()) {
|
||||
try {
|
||||
MobUtils.removeMobFiles(job.getConfiguration(), fs, tn,
|
||||
FSUtils.getTableDir(MobUtils.getMobHome(conf), tn), hcd.getName(), storeFiles);
|
||||
LOG.info(storeFiles.size() + " unused MOB files are removed");
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to archive the store files " + storeFiles, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the working directory.
|
||||
* @param job The current job.
|
||||
* @param familyName The family to cleanup
|
||||
*/
|
||||
private void cleanup(Job job, TableName tn, String familyName) {
|
||||
if (job != null) {
|
||||
// delete the working directory
|
||||
Path workingPath = new Path(job.getConfiguration().get(WORKING_DIR_KEY));
|
||||
try {
|
||||
fs.delete(workingPath, true);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to delete the working directory after sweeping store " + familyName
|
||||
+ " in the table " + tn.getNameAsString(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A result with index.
|
||||
*/
|
||||
private class IndexedResult implements Comparable<IndexedResult> {
|
||||
private int index;
|
||||
private String value;
|
||||
|
||||
public IndexedResult(int index, String value) {
|
||||
this.index = index;
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
public int getIndex() {
|
||||
return this.index;
|
||||
}
|
||||
|
||||
public String getValue() {
|
||||
return this.value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(IndexedResult o) {
|
||||
if (this.value == null && o.getValue() == null) {
|
||||
return 0;
|
||||
} else if (o.value == null) {
|
||||
return 1;
|
||||
} else if (this.value == null) {
|
||||
return -1;
|
||||
} else {
|
||||
return this.value.compareTo(o.value);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (!(obj instanceof IndexedResult)) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((IndexedResult) obj) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return value.hashCode();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge sort reader.
|
||||
* It merges and sort the readers in different sequence files as one where
|
||||
* the results are read in order.
|
||||
*/
|
||||
private class MergeSortReader {
|
||||
|
||||
private List<SequenceFile.Reader> readers = new ArrayList<SequenceFile.Reader>();
|
||||
private PriorityQueue<IndexedResult> results = new PriorityQueue<IndexedResult>();
|
||||
|
||||
public MergeSortReader(FileSystem fs, Configuration conf, Path path) throws IOException {
|
||||
if (fs.exists(path)) {
|
||||
FileStatus[] files = fs.listStatus(path);
|
||||
int index = 0;
|
||||
for (FileStatus file : files) {
|
||||
if (file.isFile()) {
|
||||
SequenceFile.Reader reader = new SequenceFile.Reader(fs, file.getPath(), conf);
|
||||
String key = (String) reader.next((String) null);
|
||||
if (key != null) {
|
||||
results.add(new IndexedResult(index, key));
|
||||
readers.add(reader);
|
||||
index++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public String next() throws IOException {
|
||||
IndexedResult result = results.poll();
|
||||
if (result != null) {
|
||||
SequenceFile.Reader reader = readers.get(result.getIndex());
|
||||
String key = (String) reader.next((String) null);
|
||||
if (key != null) {
|
||||
results.add(new IndexedResult(result.getIndex(), key));
|
||||
}
|
||||
return result.getValue();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public void close() {
|
||||
for (SequenceFile.Reader reader : readers) {
|
||||
if (reader != null) {
|
||||
IOUtils.closeStream(reader);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The counter used in sweep job.
|
||||
*/
|
||||
public enum SweepCounter {
|
||||
|
||||
/**
|
||||
* How many files are read.
|
||||
*/
|
||||
INPUT_FILE_COUNT,
|
||||
|
||||
/**
|
||||
* How many files need to be merged or cleaned.
|
||||
*/
|
||||
FILE_TO_BE_MERGE_OR_CLEAN,
|
||||
|
||||
/**
|
||||
* How many files are left after merging.
|
||||
*/
|
||||
FILE_AFTER_MERGE_OR_CLEAN,
|
||||
|
||||
/**
|
||||
* How many records are updated.
|
||||
*/
|
||||
RECORDS_UPDATED,
|
||||
}
|
||||
|
||||
public static class DummyMobAbortable implements Abortable {
|
||||
|
||||
private boolean abort = false;
|
||||
|
||||
public void abort(String why, Throwable e) {
|
||||
abort = true;
|
||||
}
|
||||
|
||||
public boolean isAborted() {
|
||||
return abort;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.mapreduce;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableLock;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* Tracker on the sweep tool node in zookeeper.
|
||||
* The sweep tool node is an ephemeral one, when the process dies this node is deleted,
|
||||
* at that time MR might be still running, and if another sweep job is started, two MR
|
||||
* for the same column family will run at the same time.
|
||||
* This tracker watches this ephemeral node, if it's gone or it's not created by the
|
||||
* sweep job that owns the current MR, the current process will be aborted.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SweepJobNodeTracker extends ZooKeeperListener {
|
||||
|
||||
private String parentNode;
|
||||
private String lockNodePrefix;
|
||||
private String owner;
|
||||
private String lockNode;
|
||||
|
||||
public SweepJobNodeTracker(ZooKeeperWatcher watcher, String parentNode, String owner) {
|
||||
super(watcher);
|
||||
this.parentNode = parentNode;
|
||||
this.owner = owner;
|
||||
this.lockNodePrefix = ZKUtil.joinZNode(parentNode, "write-");
|
||||
}
|
||||
|
||||
/**
|
||||
* Registers the watcher on the sweep job node.
|
||||
* If there's no such a sweep job node, or it's not created by the sweep job that
|
||||
* owns the current MR, the current process will be aborted.
|
||||
* This assumes the table lock uses the Zookeeper. It's a workaround and only used
|
||||
* in the sweep tool, and the sweep tool will be removed after the mob file compaction
|
||||
* is finished.
|
||||
*/
|
||||
public void start() throws KeeperException {
|
||||
watcher.registerListener(this);
|
||||
List<String> children = ZKUtil.listChildrenNoWatch(watcher, parentNode);
|
||||
if (children != null && !children.isEmpty()) {
|
||||
// there are locks
|
||||
TreeSet<String> sortedChildren = new TreeSet<String>();
|
||||
sortedChildren.addAll(children);
|
||||
// find all the write locks
|
||||
SortedSet<String> tails = sortedChildren.tailSet(lockNodePrefix);
|
||||
if (!tails.isEmpty()) {
|
||||
for (String tail : tails) {
|
||||
String path = ZKUtil.joinZNode(parentNode, tail);
|
||||
byte[] data = ZKUtil.getDataAndWatch(watcher, path);
|
||||
TableLock lock = TableLockManager.fromBytes(data);
|
||||
ServerName serverName = lock.getLockOwner();
|
||||
org.apache.hadoop.hbase.ServerName sn = org.apache.hadoop.hbase.ServerName.valueOf(
|
||||
serverName.getHostName(), serverName.getPort(), serverName.getStartCode());
|
||||
// compare the server names (host, port and start code), make sure the lock is created
|
||||
if (owner.equals(sn.toString())) {
|
||||
lockNode = path;
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void nodeDeleted(String path) {
|
||||
// If the lock node is deleted, abort the current process.
|
||||
if (path.equals(lockNode)) {
|
||||
System.exit(1);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,87 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.mapreduce;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableMapper;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.mob.mapreduce.SweepJob.DummyMobAbortable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* The mapper of a sweep job.
|
||||
* Takes the rows from the table and their results and map to <filename:Text, mobValue:KeyValue>
|
||||
* where mobValue is the actual cell in HBase.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SweepMapper extends TableMapper<Text, KeyValue> {
|
||||
|
||||
private ZooKeeperWatcher zkw = null;
|
||||
|
||||
@Override
|
||||
protected void setup(Context context) throws IOException,
|
||||
InterruptedException {
|
||||
String id = context.getConfiguration().get(SweepJob.SWEEP_JOB_ID);
|
||||
String owner = context.getConfiguration().get(SweepJob.SWEEP_JOB_SERVERNAME);
|
||||
String sweeperNode = context.getConfiguration().get(SweepJob.SWEEP_JOB_TABLE_NODE);
|
||||
zkw = new ZooKeeperWatcher(context.getConfiguration(), id,
|
||||
new DummyMobAbortable());
|
||||
try {
|
||||
SweepJobNodeTracker tracker = new SweepJobNodeTracker(zkw, sweeperNode, owner);
|
||||
tracker.start();
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void cleanup(Context context) throws IOException,
|
||||
InterruptedException {
|
||||
if (zkw != null) {
|
||||
zkw.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void map(ImmutableBytesWritable r, Result columns, Context context) throws IOException,
|
||||
InterruptedException {
|
||||
if (columns == null) {
|
||||
return;
|
||||
}
|
||||
Cell[] cells = columns.rawCells();
|
||||
if (cells == null || cells.length == 0) {
|
||||
return;
|
||||
}
|
||||
for (Cell c : cells) {
|
||||
if (MobUtils.hasValidMobRefCellValue(c)) {
|
||||
String fileName = MobUtils.getMobFileName(c);
|
||||
context.write(new Text(fileName), KeyValueUtil.ensureKeyValue(c));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,472 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.mapreduce;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutatorParams;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.mob.MobFile;
|
||||
import org.apache.hadoop.hbase.mob.MobFileName;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
|
||||
import org.apache.hadoop.hbase.mob.mapreduce.SweepJob.DummyMobAbortable;
|
||||
import org.apache.hadoop.hbase.mob.mapreduce.SweepJob.SweepCounter;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.DefaultMemStore;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.SequenceFile.CompressionType;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapreduce.Reducer;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* The reducer of a sweep job.
|
||||
* This reducer merges the small mob files into bigger ones, and write visited
|
||||
* names of mob files to a sequence file which is used by the sweep job to delete
|
||||
* the unused mob files.
|
||||
* The key of the input is a file name, the value is a collection of KeyValues
|
||||
* (the value format of KeyValue is valueLength + fileName) in HBase.
|
||||
* In this reducer, we could know how many cells exist in HBase for a mob file.
|
||||
* If the existCellSize/mobFileSize < compactionRatio, this mob
|
||||
* file needs to be merged.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SweepReducer extends Reducer<Text, KeyValue, Writable, Writable> {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(SweepReducer.class);
|
||||
|
||||
private SequenceFile.Writer writer = null;
|
||||
private MemStoreWrapper memstore;
|
||||
private Configuration conf;
|
||||
private FileSystem fs;
|
||||
|
||||
private Path familyDir;
|
||||
private CacheConfig cacheConfig;
|
||||
private long compactionBegin;
|
||||
private BufferedMutator table;
|
||||
private HColumnDescriptor family;
|
||||
private long mobCompactionDelay;
|
||||
private Path mobTableDir;
|
||||
|
||||
@Override
|
||||
protected void setup(Context context) throws IOException, InterruptedException {
|
||||
this.conf = context.getConfiguration();
|
||||
Connection c = ConnectionFactory.createConnection(this.conf);
|
||||
this.fs = FileSystem.get(conf);
|
||||
// the MOB_SWEEP_JOB_DELAY is ONE_DAY by default. Its value is only changed when testing.
|
||||
mobCompactionDelay = conf.getLong(SweepJob.MOB_SWEEP_JOB_DELAY, SweepJob.ONE_DAY);
|
||||
String tableName = conf.get(TableInputFormat.INPUT_TABLE);
|
||||
String familyName = conf.get(TableInputFormat.SCAN_COLUMN_FAMILY);
|
||||
TableName tn = TableName.valueOf(tableName);
|
||||
this.familyDir = MobUtils.getMobFamilyPath(conf, tn, familyName);
|
||||
Admin admin = c.getAdmin();
|
||||
try {
|
||||
family = admin.getTableDescriptor(tn).getFamily(Bytes.toBytes(familyName));
|
||||
if (family == null) {
|
||||
// this column family might be removed, directly return.
|
||||
throw new InvalidFamilyOperationException("Column family '" + familyName
|
||||
+ "' does not exist. It might be removed.");
|
||||
}
|
||||
} finally {
|
||||
try {
|
||||
admin.close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to close the HBaseAdmin", e);
|
||||
}
|
||||
}
|
||||
// disable the block cache.
|
||||
Configuration copyOfConf = new Configuration(conf);
|
||||
copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
|
||||
this.cacheConfig = new CacheConfig(copyOfConf);
|
||||
|
||||
table = c.getBufferedMutator(new BufferedMutatorParams(tn).writeBufferSize(1*1024*1024));
|
||||
memstore = new MemStoreWrapper(context, fs, table, family, new DefaultMemStore(), cacheConfig);
|
||||
|
||||
// The start time of the sweep tool.
|
||||
// Only the mob files whose creation time is older than startTime-oneDay will be handled by the
|
||||
// reducer since it brings inconsistency to handle the latest mob files.
|
||||
this.compactionBegin = conf.getLong(MobConstants.MOB_SWEEP_TOOL_COMPACTION_START_DATE, 0);
|
||||
mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tn);
|
||||
}
|
||||
|
||||
private SweepPartition createPartition(CompactionPartitionId id, Context context)
|
||||
throws IOException {
|
||||
return new SweepPartition(id, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(Context context) throws IOException, InterruptedException {
|
||||
String jobId = context.getConfiguration().get(SweepJob.SWEEP_JOB_ID);
|
||||
String owner = context.getConfiguration().get(SweepJob.SWEEP_JOB_SERVERNAME);
|
||||
String sweeperNode = context.getConfiguration().get(SweepJob.SWEEP_JOB_TABLE_NODE);
|
||||
ZooKeeperWatcher zkw = new ZooKeeperWatcher(context.getConfiguration(), jobId,
|
||||
new DummyMobAbortable());
|
||||
FSDataOutputStream fout = null;
|
||||
try {
|
||||
SweepJobNodeTracker tracker = new SweepJobNodeTracker(zkw, sweeperNode, owner);
|
||||
tracker.start();
|
||||
setup(context);
|
||||
// create a sequence contains all the visited file names in this reducer.
|
||||
String dir = this.conf.get(SweepJob.WORKING_VISITED_DIR_KEY);
|
||||
Path nameFilePath = new Path(dir, UUID.randomUUID().toString()
|
||||
.replace("-", MobConstants.EMPTY_STRING));
|
||||
fout = fs.create(nameFilePath, true);
|
||||
writer = SequenceFile.createWriter(context.getConfiguration(), fout, String.class,
|
||||
String.class, CompressionType.NONE, null);
|
||||
CompactionPartitionId id;
|
||||
SweepPartition partition = null;
|
||||
// the mob files which have the same start key and date are in the same partition.
|
||||
while (context.nextKey()) {
|
||||
Text key = context.getCurrentKey();
|
||||
String keyString = key.toString();
|
||||
id = createPartitionId(keyString);
|
||||
if (null == partition || !id.equals(partition.getId())) {
|
||||
// It's the first mob file in the current partition.
|
||||
if (null != partition) {
|
||||
// this mob file is in different partitions with the previous mob file.
|
||||
// directly close.
|
||||
partition.close();
|
||||
}
|
||||
// create a new one
|
||||
partition = createPartition(id, context);
|
||||
}
|
||||
if (partition != null) {
|
||||
// run the partition
|
||||
partition.execute(key, context.getValues());
|
||||
}
|
||||
}
|
||||
if (null != partition) {
|
||||
partition.close();
|
||||
}
|
||||
writer.hflush();
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
} finally {
|
||||
cleanup(context);
|
||||
zkw.close();
|
||||
if (writer != null) {
|
||||
IOUtils.closeStream(writer);
|
||||
}
|
||||
if (fout != null) {
|
||||
IOUtils.closeStream(fout);
|
||||
}
|
||||
if (table != null) {
|
||||
try {
|
||||
table.close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* The mob files which have the same start key and date are in the same partition.
|
||||
* The files in the same partition are merged together into bigger ones.
|
||||
*/
|
||||
public class SweepPartition {
|
||||
|
||||
private final CompactionPartitionId id;
|
||||
private final Context context;
|
||||
private boolean memstoreUpdated = false;
|
||||
private boolean mergeSmall = false;
|
||||
private final Map<String, MobFileStatus> fileStatusMap = new HashMap<String, MobFileStatus>();
|
||||
private final List<Path> toBeDeleted = new ArrayList<Path>();
|
||||
|
||||
public SweepPartition(CompactionPartitionId id, Context context) throws IOException {
|
||||
this.id = id;
|
||||
this.context = context;
|
||||
memstore.setPartitionId(id);
|
||||
init();
|
||||
}
|
||||
|
||||
public CompactionPartitionId getId() {
|
||||
return this.id;
|
||||
}
|
||||
|
||||
/**
|
||||
* Prepares the map of files.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
private void init() throws IOException {
|
||||
FileStatus[] fileStats = listStatus(familyDir, id.getStartKey());
|
||||
if (null == fileStats) {
|
||||
return;
|
||||
}
|
||||
|
||||
int smallFileCount = 0;
|
||||
float compactionRatio = conf.getFloat(MobConstants.MOB_SWEEP_TOOL_COMPACTION_RATIO,
|
||||
MobConstants.DEFAULT_SWEEP_TOOL_MOB_COMPACTION_RATIO);
|
||||
long compactionMergeableSize = conf.getLong(
|
||||
MobConstants.MOB_SWEEP_TOOL_COMPACTION_MERGEABLE_SIZE,
|
||||
MobConstants.DEFAULT_SWEEP_TOOL_MOB_COMPACTION_MERGEABLE_SIZE);
|
||||
// list the files. Just merge the hfiles, don't merge the hfile links.
|
||||
// prepare the map of mob files. The key is the file name, the value is the file status.
|
||||
for (FileStatus fileStat : fileStats) {
|
||||
MobFileStatus mobFileStatus = null;
|
||||
if (!HFileLink.isHFileLink(fileStat.getPath())) {
|
||||
mobFileStatus = new MobFileStatus(fileStat, compactionRatio, compactionMergeableSize);
|
||||
if (mobFileStatus.needMerge()) {
|
||||
smallFileCount++;
|
||||
}
|
||||
// key is file name (not hfile name), value is hfile status.
|
||||
fileStatusMap.put(fileStat.getPath().getName(), mobFileStatus);
|
||||
}
|
||||
}
|
||||
if (smallFileCount >= 2) {
|
||||
// merge the files only when there're more than 1 files in the same partition.
|
||||
this.mergeSmall = true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Flushes the data into mob files and store files, and archives the small
|
||||
* files after they're merged.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
if (null == id) {
|
||||
return;
|
||||
}
|
||||
// flush remain key values into mob files
|
||||
if (memstoreUpdated) {
|
||||
memstore.flushMemStore();
|
||||
}
|
||||
List<StoreFile> storeFiles = new ArrayList<StoreFile>(toBeDeleted.size());
|
||||
// delete samll files after compaction
|
||||
for (Path path : toBeDeleted) {
|
||||
LOG.info("[In Partition close] Delete the file " + path + " in partition close");
|
||||
storeFiles.add(new StoreFile(fs, path, conf, cacheConfig, BloomType.NONE));
|
||||
}
|
||||
if (!storeFiles.isEmpty()) {
|
||||
try {
|
||||
MobUtils.removeMobFiles(conf, fs, table.getName(), mobTableDir, family.getName(),
|
||||
storeFiles);
|
||||
context.getCounter(SweepCounter.FILE_TO_BE_MERGE_OR_CLEAN).increment(storeFiles.size());
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to archive the store files " + storeFiles, e);
|
||||
}
|
||||
storeFiles.clear();
|
||||
}
|
||||
fileStatusMap.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Merges the small mob files into bigger ones.
|
||||
* @param fileName The current mob file name.
|
||||
* @param values The collection of KeyValues in this mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void execute(Text fileName, Iterable<KeyValue> values) throws IOException {
|
||||
if (null == values) {
|
||||
return;
|
||||
}
|
||||
MobFileName mobFileName = MobFileName.create(fileName.toString());
|
||||
LOG.info("[In reducer] The file name: " + fileName.toString());
|
||||
MobFileStatus mobFileStat = fileStatusMap.get(mobFileName.getFileName());
|
||||
if (null == mobFileStat) {
|
||||
LOG.info("[In reducer] Cannot find the file, probably this record is obsolete");
|
||||
return;
|
||||
}
|
||||
// only handle the files that are older then one day.
|
||||
if (compactionBegin - mobFileStat.getFileStatus().getModificationTime()
|
||||
<= mobCompactionDelay) {
|
||||
return;
|
||||
}
|
||||
// write the hfile name
|
||||
writer.append(mobFileName.getFileName(), MobConstants.EMPTY_STRING);
|
||||
Set<KeyValue> kvs = new HashSet<KeyValue>();
|
||||
for (KeyValue kv : values) {
|
||||
if (kv.getValueLength() > Bytes.SIZEOF_INT) {
|
||||
mobFileStat.addValidSize(Bytes.toInt(kv.getValueArray(), kv.getValueOffset(),
|
||||
Bytes.SIZEOF_INT));
|
||||
}
|
||||
kvs.add(kv.createKeyOnly(false));
|
||||
}
|
||||
// If the mob file is a invalid one or a small one, merge it into new/bigger ones.
|
||||
if (mobFileStat.needClean() || (mergeSmall && mobFileStat.needMerge())) {
|
||||
context.getCounter(SweepCounter.INPUT_FILE_COUNT).increment(1);
|
||||
MobFile file = MobFile.create(fs,
|
||||
new Path(familyDir, mobFileName.getFileName()), conf, cacheConfig);
|
||||
StoreFileScanner scanner = null;
|
||||
file.open();
|
||||
try {
|
||||
scanner = file.getScanner();
|
||||
scanner.seek(KeyValueUtil.createFirstOnRow(HConstants.EMPTY_BYTE_ARRAY));
|
||||
Cell cell;
|
||||
while (null != (cell = scanner.next())) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
KeyValue keyOnly = kv.createKeyOnly(false);
|
||||
if (kvs.contains(keyOnly)) {
|
||||
// write the KeyValue existing in HBase to the memstore.
|
||||
memstore.addToMemstore(kv);
|
||||
memstoreUpdated = true;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (scanner != null) {
|
||||
scanner.close();
|
||||
}
|
||||
file.close();
|
||||
}
|
||||
toBeDeleted.add(mobFileStat.getFileStatus().getPath());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Lists the files with the same prefix.
|
||||
* @param p The file path.
|
||||
* @param prefix The prefix.
|
||||
* @return The files with the same prefix.
|
||||
* @throws IOException
|
||||
*/
|
||||
private FileStatus[] listStatus(Path p, String prefix) throws IOException {
|
||||
return fs.listStatus(p, new PathPrefixFilter(prefix));
|
||||
}
|
||||
}
|
||||
|
||||
static class PathPrefixFilter implements PathFilter {
|
||||
|
||||
private final String prefix;
|
||||
|
||||
public PathPrefixFilter(String prefix) {
|
||||
this.prefix = prefix;
|
||||
}
|
||||
|
||||
public boolean accept(Path path) {
|
||||
return path.getName().startsWith(prefix, 0);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the partition id.
|
||||
* @param fileNameAsString The current file name, in string.
|
||||
* @return The partition id.
|
||||
*/
|
||||
private CompactionPartitionId createPartitionId(String fileNameAsString) {
|
||||
MobFileName fileName = MobFileName.create(fileNameAsString);
|
||||
return new CompactionPartitionId(fileName.getStartKey(), fileName.getDate());
|
||||
}
|
||||
|
||||
/**
|
||||
* The mob file status used in the sweep reduecer.
|
||||
*/
|
||||
private static class MobFileStatus {
|
||||
private FileStatus fileStatus;
|
||||
private int validSize;
|
||||
private long size;
|
||||
|
||||
private float compactionRatio = MobConstants.DEFAULT_SWEEP_TOOL_MOB_COMPACTION_RATIO;
|
||||
private long compactionMergeableSize =
|
||||
MobConstants.DEFAULT_SWEEP_TOOL_MOB_COMPACTION_MERGEABLE_SIZE;
|
||||
|
||||
/**
|
||||
* @param fileStatus The current FileStatus.
|
||||
* @param compactionRatio compactionRatio the invalid ratio.
|
||||
* If there're too many cells deleted in a mob file, it's regarded as invalid,
|
||||
* and needs to be written to a new one.
|
||||
* If existingCellSize/fileSize < compactionRatio, it's regarded as a invalid one.
|
||||
* @param compactionMergeableSize compactionMergeableSize If the size of a mob file is less
|
||||
* than this value, it's regarded as a small file and needs to be merged
|
||||
*/
|
||||
public MobFileStatus(FileStatus fileStatus, float compactionRatio,
|
||||
long compactionMergeableSize) {
|
||||
this.fileStatus = fileStatus;
|
||||
this.size = fileStatus.getLen();
|
||||
validSize = 0;
|
||||
this.compactionRatio = compactionRatio;
|
||||
this.compactionMergeableSize = compactionMergeableSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add size to this file.
|
||||
* @param size The size to be added.
|
||||
*/
|
||||
public void addValidSize(int size) {
|
||||
this.validSize += size;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether the mob files need to be cleaned.
|
||||
* If there're too many cells deleted in this mob file, it needs to be cleaned.
|
||||
* @return True if it needs to be cleaned.
|
||||
*/
|
||||
public boolean needClean() {
|
||||
return validSize < compactionRatio * size;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether the mob files need to be merged.
|
||||
* If this mob file is too small, it needs to be merged.
|
||||
* @return True if it needs to be merged.
|
||||
*/
|
||||
public boolean needMerge() {
|
||||
return this.size < compactionMergeableSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the file status.
|
||||
* @return The file status.
|
||||
*/
|
||||
public FileStatus getFileStatus() {
|
||||
return fileStatus;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,119 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.mapreduce;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* The sweep tool. It deletes the mob files that are not used and merges the small mob files to
|
||||
* bigger ones. Each run of this sweep tool only handles one column family. The runs on
|
||||
* the same column family are mutually exclusive. And the major compaction and sweep tool on the
|
||||
* same column family are mutually exclusive too.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class Sweeper extends Configured implements Tool {
|
||||
|
||||
/**
|
||||
* Sweeps the mob files on one column family. It deletes the unused mob files and merges
|
||||
* the small mob files into bigger ones.
|
||||
* @param tableName The current table name in string format.
|
||||
* @param familyName The column family name.
|
||||
* @return 0 if success, 2 if job aborted with an exception, 3 if unable to start due to
|
||||
* other compaction,4 if mr job was unsuccessful
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
* @throws ClassNotFoundException
|
||||
* @throws KeeperException
|
||||
* @throws ServiceException
|
||||
*/
|
||||
int sweepFamily(String tableName, String familyName) throws IOException, InterruptedException,
|
||||
ClassNotFoundException, KeeperException, ServiceException {
|
||||
Configuration conf = getConf();
|
||||
// make sure the target HBase exists.
|
||||
HBaseAdmin.checkHBaseAvailable(conf);
|
||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
||||
try {
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
TableName tn = TableName.valueOf(tableName);
|
||||
HTableDescriptor htd = admin.getTableDescriptor(tn);
|
||||
HColumnDescriptor family = htd.getFamily(Bytes.toBytes(familyName));
|
||||
if (family == null || !family.isMobEnabled()) {
|
||||
throw new IOException("Column family " + familyName + " is not a MOB column family");
|
||||
}
|
||||
SweepJob job = new SweepJob(conf, fs);
|
||||
// Run the sweeping
|
||||
return job.sweep(tn, family);
|
||||
} catch (Exception e) {
|
||||
System.err.println("Job aborted due to exception " + e);
|
||||
return 2; // job failed
|
||||
} finally {
|
||||
try {
|
||||
admin.close();
|
||||
} catch (IOException e) {
|
||||
System.out.println("Failed to close the HBaseAdmin: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
int ret = ToolRunner.run(conf, new Sweeper(), args);
|
||||
System.exit(ret);
|
||||
}
|
||||
|
||||
private void printUsage() {
|
||||
System.err.println("Usage:\n" + "--------------------------\n" + Sweeper.class.getName()
|
||||
+ " tableName familyName");
|
||||
System.err.println(" tableName The table name");
|
||||
System.err.println(" familyName The column family name");
|
||||
}
|
||||
|
||||
/**
|
||||
* Main method for the tool.
|
||||
* @return 0 if success, 1 for bad args. 2 if job aborted with an exception,
|
||||
* 3 if unable to start due to other compaction, 4 if mr job was unsuccessful
|
||||
*/
|
||||
public int run(String[] args) throws Exception {
|
||||
if (args.length != 2) {
|
||||
printUsage();
|
||||
return 1;
|
||||
}
|
||||
String table = args[0];
|
||||
String family = args[1];
|
||||
return sweepFamily(table, family);
|
||||
}
|
||||
}
|
|
@ -64,6 +64,14 @@ public class DefaultStoreEngine extends StoreEngine<
|
|||
@Override
|
||||
protected void createComponents(
|
||||
Configuration conf, Store store, CellComparator kvComparator) throws IOException {
|
||||
createCompactor(conf, store);
|
||||
createCompactionPolicy(conf, store);
|
||||
createStoreFlusher(conf, store);
|
||||
storeFileManager = new DefaultStoreFileManager(kvComparator, conf, compactionPolicy.getConf());
|
||||
|
||||
}
|
||||
|
||||
protected void createCompactor(Configuration conf, Store store) throws IOException {
|
||||
String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
|
||||
try {
|
||||
compactor = ReflectionUtils.instantiateWithCustomCtor(className,
|
||||
|
@ -71,7 +79,10 @@ public class DefaultStoreEngine extends StoreEngine<
|
|||
} catch (Exception e) {
|
||||
throw new IOException("Unable to load configured compactor '" + className + "'", e);
|
||||
}
|
||||
className = conf.get(
|
||||
}
|
||||
|
||||
protected void createCompactionPolicy(Configuration conf, Store store) throws IOException {
|
||||
String className = conf.get(
|
||||
DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
|
||||
try {
|
||||
compactionPolicy = ReflectionUtils.instantiateWithCustomCtor(className,
|
||||
|
@ -80,8 +91,10 @@ public class DefaultStoreEngine extends StoreEngine<
|
|||
} catch (Exception e) {
|
||||
throw new IOException("Unable to load configured compaction policy '" + className + "'", e);
|
||||
}
|
||||
storeFileManager = new DefaultStoreFileManager(kvComparator, conf, compactionPolicy.getConf());
|
||||
className = conf.get(
|
||||
}
|
||||
|
||||
protected void createStoreFlusher(Configuration conf, Store store) throws IOException {
|
||||
String className = conf.get(
|
||||
DEFAULT_STORE_FLUSHER_CLASS_KEY, DEFAULT_STORE_FLUSHER_CLASS.getName());
|
||||
try {
|
||||
storeFlusher = ReflectionUtils.instantiateWithCustomCtor(className,
|
||||
|
@ -91,7 +104,6 @@ public class DefaultStoreEngine extends StoreEngine<
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public CompactionContext createCompaction() {
|
||||
return new DefaultCompactionContext();
|
||||
|
|
|
@ -85,7 +85,7 @@ public class DefaultStoreFlusher extends StoreFlusher {
|
|||
scanner.close();
|
||||
}
|
||||
LOG.info("Flushed, sequenceid=" + cacheFlushId +", memsize="
|
||||
+ StringUtils.humanReadableInt(snapshot.getSize()) +
|
||||
+ StringUtils.TraditionalBinaryPrefix.long2String(snapshot.getSize(), "", 1) +
|
||||
", hasBloomFilter=" + writer.hasGeneralBloom() +
|
||||
", into tmp file " + writer.getPath());
|
||||
result.add(writer.getPath());
|
||||
|
|
|
@ -0,0 +1,585 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.FilterList;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.mob.MobCacheConfig;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.mob.MobFile;
|
||||
import org.apache.hadoop.hbase.mob.MobFileName;
|
||||
import org.apache.hadoop.hbase.mob.MobStoreEngine;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
import org.apache.hadoop.hbase.util.IdLock;
|
||||
|
||||
/**
|
||||
* The store implementation to save MOBs (medium objects), it extends the HStore.
|
||||
* When a descriptor of a column family has the value "IS_MOB", it means this column family
|
||||
* is a mob one. When a HRegion instantiate a store for this column family, the HMobStore is
|
||||
* created.
|
||||
* HMobStore is almost the same with the HStore except using different types of scanners.
|
||||
* In the method of getScanner, the MobStoreScanner and MobReversedStoreScanner are returned.
|
||||
* In these scanners, a additional seeks in the mob files should be performed after the seek
|
||||
* to HBase is done.
|
||||
* The store implements how we save MOBs by extending HStore. When a descriptor
|
||||
* of a column family has the value "IS_MOB", it means this column family is a mob one. When a
|
||||
* HRegion instantiate a store for this column family, the HMobStore is created. HMobStore is
|
||||
* almost the same with the HStore except using different types of scanners. In the method of
|
||||
* getScanner, the MobStoreScanner and MobReversedStoreScanner are returned. In these scanners, a
|
||||
* additional seeks in the mob files should be performed after the seek in HBase is done.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HMobStore extends HStore {
|
||||
private static final Log LOG = LogFactory.getLog(HMobStore.class);
|
||||
private MobCacheConfig mobCacheConfig;
|
||||
private Path homePath;
|
||||
private Path mobFamilyPath;
|
||||
private volatile long cellsCountCompactedToMob = 0;
|
||||
private volatile long cellsCountCompactedFromMob = 0;
|
||||
private volatile long cellsSizeCompactedToMob = 0;
|
||||
private volatile long cellsSizeCompactedFromMob = 0;
|
||||
private volatile long mobFlushCount = 0;
|
||||
private volatile long mobFlushedCellsCount = 0;
|
||||
private volatile long mobFlushedCellsSize = 0;
|
||||
private volatile long mobScanCellsCount = 0;
|
||||
private volatile long mobScanCellsSize = 0;
|
||||
private HColumnDescriptor family;
|
||||
private TableLockManager tableLockManager;
|
||||
private TableName tableLockName;
|
||||
private Map<String, List<Path>> map = new ConcurrentHashMap<String, List<Path>>();
|
||||
private final IdLock keyLock = new IdLock();
|
||||
|
||||
public HMobStore(final HRegion region, final HColumnDescriptor family,
|
||||
final Configuration confParam) throws IOException {
|
||||
super(region, family, confParam);
|
||||
this.family = family;
|
||||
this.mobCacheConfig = (MobCacheConfig) cacheConf;
|
||||
this.homePath = MobUtils.getMobHome(conf);
|
||||
this.mobFamilyPath = MobUtils.getMobFamilyPath(conf, this.getTableName(),
|
||||
family.getNameAsString());
|
||||
List<Path> locations = new ArrayList<Path>(2);
|
||||
locations.add(mobFamilyPath);
|
||||
TableName tn = region.getTableDesc().getTableName();
|
||||
locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils.getMobRegionInfo(tn)
|
||||
.getEncodedName(), family.getNameAsString()));
|
||||
map.put(Bytes.toString(tn.getName()), locations);
|
||||
if (region.getRegionServerServices() != null) {
|
||||
tableLockManager = region.getRegionServerServices().getTableLockManager();
|
||||
tableLockName = MobUtils.getTableLockName(getTableName());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the mob cache config.
|
||||
*/
|
||||
@Override
|
||||
protected void createCacheConf(HColumnDescriptor family) {
|
||||
cacheConf = new MobCacheConfig(conf, family);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets current config.
|
||||
*/
|
||||
public Configuration getConfiguration() {
|
||||
return this.conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the MobStoreScanner or MobReversedStoreScanner. In these scanners, a additional seeks in
|
||||
* the mob files should be performed after the seek in HBase is done.
|
||||
*/
|
||||
@Override
|
||||
protected KeyValueScanner createScanner(Scan scan, final NavigableSet<byte[]> targetCols,
|
||||
long readPt, KeyValueScanner scanner) throws IOException {
|
||||
if (scanner == null) {
|
||||
if (MobUtils.isRefOnlyScan(scan)) {
|
||||
Filter refOnlyFilter = new MobReferenceOnlyFilter();
|
||||
Filter filter = scan.getFilter();
|
||||
if (filter != null) {
|
||||
scan.setFilter(new FilterList(filter, refOnlyFilter));
|
||||
} else {
|
||||
scan.setFilter(refOnlyFilter);
|
||||
}
|
||||
}
|
||||
scanner = scan.isReversed() ? new ReversedMobStoreScanner(this, getScanInfo(), scan,
|
||||
targetCols, readPt) : new MobStoreScanner(this, getScanInfo(), scan, targetCols, readPt);
|
||||
}
|
||||
return scanner;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the mob store engine.
|
||||
*/
|
||||
@Override
|
||||
protected StoreEngine<?, ?, ?, ?> createStoreEngine(Store store, Configuration conf,
|
||||
CellComparator cellComparator) throws IOException {
|
||||
MobStoreEngine engine = new MobStoreEngine();
|
||||
engine.createComponents(conf, store, cellComparator);
|
||||
return engine;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the temp directory.
|
||||
* @return The temp directory.
|
||||
*/
|
||||
private Path getTempDir() {
|
||||
return new Path(homePath, MobConstants.TEMP_DIR_NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the writer for the mob file in temp directory.
|
||||
* @param date The latest date of written cells.
|
||||
* @param maxKeyCount The key count.
|
||||
* @param compression The compression algorithm.
|
||||
* @param startKey The start key.
|
||||
* @return The writer for the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public StoreFile.Writer createWriterInTmp(Date date, long maxKeyCount,
|
||||
Compression.Algorithm compression, byte[] startKey) throws IOException {
|
||||
if (startKey == null) {
|
||||
startKey = HConstants.EMPTY_START_ROW;
|
||||
}
|
||||
Path path = getTempDir();
|
||||
return createWriterInTmp(MobUtils.formatDate(date), path, maxKeyCount, compression, startKey);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the writer for the del file in temp directory.
|
||||
* The del file keeps tracking the delete markers. Its name has a suffix _del,
|
||||
* the format is [0-9a-f]+(_del)?.
|
||||
* @param date The latest date of written cells.
|
||||
* @param maxKeyCount The key count.
|
||||
* @param compression The compression algorithm.
|
||||
* @param startKey The start key.
|
||||
* @return The writer for the del file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public StoreFile.Writer createDelFileWriterInTmp(Date date, long maxKeyCount,
|
||||
Compression.Algorithm compression, byte[] startKey) throws IOException {
|
||||
if (startKey == null) {
|
||||
startKey = HConstants.EMPTY_START_ROW;
|
||||
}
|
||||
Path path = getTempDir();
|
||||
String suffix = UUID
|
||||
.randomUUID().toString().replaceAll("-", "") + "_del";
|
||||
MobFileName mobFileName = MobFileName.create(startKey, MobUtils.formatDate(date), suffix);
|
||||
return createWriterInTmp(mobFileName, path, maxKeyCount, compression);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the writer for the mob file in temp directory.
|
||||
* @param date The date string, its format is yyyymmmdd.
|
||||
* @param basePath The basic path for a temp directory.
|
||||
* @param maxKeyCount The key count.
|
||||
* @param compression The compression algorithm.
|
||||
* @param startKey The start key.
|
||||
* @return The writer for the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public StoreFile.Writer createWriterInTmp(String date, Path basePath, long maxKeyCount,
|
||||
Compression.Algorithm compression, byte[] startKey) throws IOException {
|
||||
MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID()
|
||||
.toString().replaceAll("-", ""));
|
||||
return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the writer for the mob file in temp directory.
|
||||
* @param mobFileName The mob file name.
|
||||
* @param basePath The basic path for a temp directory.
|
||||
* @param maxKeyCount The key count.
|
||||
* @param compression The compression algorithm.
|
||||
* @return The writer for the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public StoreFile.Writer createWriterInTmp(MobFileName mobFileName, Path basePath,
|
||||
long maxKeyCount, Compression.Algorithm compression) throws IOException {
|
||||
final CacheConfig writerCacheConf = mobCacheConfig;
|
||||
HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
|
||||
.withIncludesMvcc(true).withIncludesTags(true)
|
||||
.withCompressTags(family.isCompressTags())
|
||||
.withChecksumType(checksumType)
|
||||
.withBytesPerCheckSum(bytesPerChecksum)
|
||||
.withBlockSize(blocksize)
|
||||
.withHBaseCheckSum(true).withDataBlockEncoding(getFamily().getDataBlockEncoding())
|
||||
.withEncryptionContext(cryptoContext)
|
||||
.withCreateTime(EnvironmentEdgeManager.currentTime()).build();
|
||||
|
||||
StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf, region.getFilesystem())
|
||||
.withFilePath(new Path(basePath, mobFileName.getFileName()))
|
||||
.withComparator(CellComparator.COMPARATOR).withBloomType(BloomType.NONE)
|
||||
.withMaxKeyCount(maxKeyCount).withFileContext(hFileContext).build();
|
||||
return w;
|
||||
}
|
||||
|
||||
/**
|
||||
* Commits the mob file.
|
||||
* @param sourceFile The source file.
|
||||
* @param targetPath The directory path where the source file is renamed to.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void commitFile(final Path sourceFile, Path targetPath) throws IOException {
|
||||
if (sourceFile == null) {
|
||||
return;
|
||||
}
|
||||
Path dstPath = new Path(targetPath, sourceFile.getName());
|
||||
validateMobFile(sourceFile);
|
||||
String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
|
||||
LOG.info(msg);
|
||||
Path parent = dstPath.getParent();
|
||||
if (!region.getFilesystem().exists(parent)) {
|
||||
region.getFilesystem().mkdirs(parent);
|
||||
}
|
||||
if (!region.getFilesystem().rename(sourceFile, dstPath)) {
|
||||
throw new IOException("Failed rename of " + sourceFile + " to " + dstPath);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates a mob file by opening and closing it.
|
||||
*
|
||||
* @param path the path to the mob file
|
||||
*/
|
||||
private void validateMobFile(Path path) throws IOException {
|
||||
StoreFile storeFile = null;
|
||||
try {
|
||||
storeFile =
|
||||
new StoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig, BloomType.NONE);
|
||||
storeFile.createReader();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Fail to open mob file[" + path + "], keep it in temp directory.", e);
|
||||
throw e;
|
||||
} finally {
|
||||
if (storeFile != null) {
|
||||
storeFile.closeReader(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the cell from the mob file, and the read point does not count.
|
||||
* This is used for DefaultMobStoreCompactor where we can read empty value for the missing cell.
|
||||
* @param reference The cell found in the HBase, its value is a path to a mob file.
|
||||
* @param cacheBlocks Whether the scanner should cache blocks.
|
||||
* @return The cell found in the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Cell resolve(Cell reference, boolean cacheBlocks) throws IOException {
|
||||
return resolve(reference, cacheBlocks, -1, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the cell from the mob file.
|
||||
* @param reference The cell found in the HBase, its value is a path to a mob file.
|
||||
* @param cacheBlocks Whether the scanner should cache blocks.
|
||||
* @param readPt the read point.
|
||||
* @param readEmptyValueOnMobCellMiss Whether return null value when the mob file is
|
||||
* missing or corrupt.
|
||||
* @return The cell found in the mob file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Cell resolve(Cell reference, boolean cacheBlocks, long readPt,
|
||||
boolean readEmptyValueOnMobCellMiss) throws IOException {
|
||||
Cell result = null;
|
||||
if (MobUtils.hasValidMobRefCellValue(reference)) {
|
||||
String fileName = MobUtils.getMobFileName(reference);
|
||||
Tag tableNameTag = MobUtils.getTableNameTag(reference);
|
||||
if (tableNameTag != null) {
|
||||
byte[] tableName = tableNameTag.getValue();
|
||||
String tableNameString = Bytes.toString(tableName);
|
||||
List<Path> locations = map.get(tableNameString);
|
||||
if (locations == null) {
|
||||
IdLock.Entry lockEntry = keyLock.getLockEntry(tableNameString.hashCode());
|
||||
try {
|
||||
locations = map.get(tableNameString);
|
||||
if (locations == null) {
|
||||
locations = new ArrayList<Path>(2);
|
||||
TableName tn = TableName.valueOf(tableName);
|
||||
locations.add(MobUtils.getMobFamilyPath(conf, tn, family.getNameAsString()));
|
||||
locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils
|
||||
.getMobRegionInfo(tn).getEncodedName(), family.getNameAsString()));
|
||||
map.put(tableNameString, locations);
|
||||
}
|
||||
} finally {
|
||||
keyLock.releaseLockEntry(lockEntry);
|
||||
}
|
||||
}
|
||||
result = readCell(locations, fileName, reference, cacheBlocks, readPt,
|
||||
readEmptyValueOnMobCellMiss);
|
||||
}
|
||||
}
|
||||
if (result == null) {
|
||||
LOG.warn("The KeyValue result is null, assemble a new KeyValue with the same row,family,"
|
||||
+ "qualifier,timestamp,type and tags but with an empty value to return.");
|
||||
result = new KeyValue(reference.getRowArray(), reference.getRowOffset(),
|
||||
reference.getRowLength(), reference.getFamilyArray(), reference.getFamilyOffset(),
|
||||
reference.getFamilyLength(), reference.getQualifierArray(),
|
||||
reference.getQualifierOffset(), reference.getQualifierLength(), reference.getTimestamp(),
|
||||
Type.codeToType(reference.getTypeByte()), HConstants.EMPTY_BYTE_ARRAY,
|
||||
0, 0, reference.getTagsArray(), reference.getTagsOffset(),
|
||||
reference.getTagsLength());
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the cell from a mob file.
|
||||
* The mob file might be located in different directories.
|
||||
* 1. The working directory.
|
||||
* 2. The archive directory.
|
||||
* Reads the cell from the files located in both of the above directories.
|
||||
* @param locations The possible locations where the mob files are saved.
|
||||
* @param fileName The file to be read.
|
||||
* @param search The cell to be searched.
|
||||
* @param cacheMobBlocks Whether the scanner should cache blocks.
|
||||
* @param readPt the read point.
|
||||
* @param readEmptyValueOnMobCellMiss Whether return null value when the mob file is
|
||||
* missing or corrupt.
|
||||
* @return The found cell. Null if there's no such a cell.
|
||||
* @throws IOException
|
||||
*/
|
||||
private Cell readCell(List<Path> locations, String fileName, Cell search, boolean cacheMobBlocks,
|
||||
long readPt, boolean readEmptyValueOnMobCellMiss) throws IOException {
|
||||
FileSystem fs = getFileSystem();
|
||||
Throwable throwable = null;
|
||||
for (Path location : locations) {
|
||||
MobFile file = null;
|
||||
Path path = new Path(location, fileName);
|
||||
try {
|
||||
file = mobCacheConfig.getMobFileCache().openFile(fs, path, mobCacheConfig);
|
||||
return readPt != -1 ? file.readCell(search, cacheMobBlocks, readPt) : file.readCell(search,
|
||||
cacheMobBlocks);
|
||||
} catch (IOException e) {
|
||||
mobCacheConfig.getMobFileCache().evictFile(fileName);
|
||||
throwable = e;
|
||||
if ((e instanceof FileNotFoundException) ||
|
||||
(e.getCause() instanceof FileNotFoundException)) {
|
||||
LOG.warn("Fail to read the cell, the mob file " + path + " doesn't exist", e);
|
||||
} else if (e instanceof CorruptHFileException) {
|
||||
LOG.error("The mob file " + path + " is corrupt", e);
|
||||
break;
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
} catch (NullPointerException e) { // HDFS 1.x - DFSInputStream.getBlockAt()
|
||||
mobCacheConfig.getMobFileCache().evictFile(fileName);
|
||||
LOG.warn("Fail to read the cell", e);
|
||||
throwable = e;
|
||||
} catch (AssertionError e) { // assert in HDFS 1.x - DFSInputStream.getBlockAt()
|
||||
mobCacheConfig.getMobFileCache().evictFile(fileName);
|
||||
LOG.warn("Fail to read the cell", e);
|
||||
throwable = e;
|
||||
} finally {
|
||||
if (file != null) {
|
||||
mobCacheConfig.getMobFileCache().closeFile(file);
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG.error("The mob file " + fileName + " could not be found in the locations " + locations
|
||||
+ " or it is corrupt");
|
||||
if (readEmptyValueOnMobCellMiss) {
|
||||
return null;
|
||||
} else if (throwable instanceof IOException) {
|
||||
throw (IOException) throwable;
|
||||
} else {
|
||||
throw new IOException(throwable);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the mob file path.
|
||||
* @return The mob file path.
|
||||
*/
|
||||
public Path getPath() {
|
||||
return mobFamilyPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* The compaction in the store of mob.
|
||||
* The cells in this store contains the path of the mob files. There might be race
|
||||
* condition between the major compaction and the sweeping in mob files.
|
||||
* In order to avoid this, we need mutually exclude the running of the major compaction and
|
||||
* sweeping in mob files.
|
||||
* The minor compaction is not affected.
|
||||
* The major compaction is marked as retainDeleteMarkers when a sweeping is in progress.
|
||||
*/
|
||||
@Override
|
||||
public List<StoreFile> compact(CompactionContext compaction,
|
||||
CompactionThroughputController throughputController) throws IOException {
|
||||
// If it's major compaction, try to find whether there's a sweeper is running
|
||||
// If yes, mark the major compaction as retainDeleteMarkers
|
||||
if (compaction.getRequest().isAllFiles()) {
|
||||
// Use the Zookeeper to coordinate.
|
||||
// 1. Acquire a operation lock.
|
||||
// 1.1. If no, mark the major compaction as retainDeleteMarkers and continue the compaction.
|
||||
// 1.2. If the lock is obtained, search the node of sweeping.
|
||||
// 1.2.1. If the node is there, the sweeping is in progress, mark the major
|
||||
// compaction as retainDeleteMarkers and continue the compaction.
|
||||
// 1.2.2. If the node is not there, add a child to the major compaction node, and
|
||||
// run the compaction directly.
|
||||
TableLock lock = null;
|
||||
if (tableLockManager != null) {
|
||||
lock = tableLockManager.readLock(tableLockName, "Major compaction in HMobStore");
|
||||
}
|
||||
boolean tableLocked = false;
|
||||
String tableName = getTableName().getNameAsString();
|
||||
if (lock != null) {
|
||||
try {
|
||||
LOG.info("Start to acquire a read lock for the table[" + tableName
|
||||
+ "], ready to perform the major compaction");
|
||||
lock.acquire();
|
||||
tableLocked = true;
|
||||
} catch (Exception e) {
|
||||
LOG.error("Fail to lock the table " + tableName, e);
|
||||
}
|
||||
} else {
|
||||
// If the tableLockManager is null, mark the tableLocked as true.
|
||||
tableLocked = true;
|
||||
}
|
||||
try {
|
||||
if (!tableLocked) {
|
||||
LOG.warn("Cannot obtain the table lock, maybe a sweep tool is running on this table["
|
||||
+ tableName + "], forcing the delete markers to be retained");
|
||||
compaction.getRequest().forceRetainDeleteMarkers();
|
||||
}
|
||||
return super.compact(compaction, throughputController);
|
||||
} finally {
|
||||
if (tableLocked && lock != null) {
|
||||
try {
|
||||
lock.release();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Fail to release the table lock " + tableName, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// If it's not a major compaction, continue the compaction.
|
||||
return super.compact(compaction, throughputController);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateCellsCountCompactedToMob(long count) {
|
||||
cellsCountCompactedToMob += count;
|
||||
}
|
||||
|
||||
public long getCellsCountCompactedToMob() {
|
||||
return cellsCountCompactedToMob;
|
||||
}
|
||||
|
||||
public void updateCellsCountCompactedFromMob(long count) {
|
||||
cellsCountCompactedFromMob += count;
|
||||
}
|
||||
|
||||
public long getCellsCountCompactedFromMob() {
|
||||
return cellsCountCompactedFromMob;
|
||||
}
|
||||
|
||||
public void updateCellsSizeCompactedToMob(long size) {
|
||||
cellsSizeCompactedToMob += size;
|
||||
}
|
||||
|
||||
public long getCellsSizeCompactedToMob() {
|
||||
return cellsSizeCompactedToMob;
|
||||
}
|
||||
|
||||
public void updateCellsSizeCompactedFromMob(long size) {
|
||||
cellsSizeCompactedFromMob += size;
|
||||
}
|
||||
|
||||
public long getCellsSizeCompactedFromMob() {
|
||||
return cellsSizeCompactedFromMob;
|
||||
}
|
||||
|
||||
public void updateMobFlushCount() {
|
||||
mobFlushCount++;
|
||||
}
|
||||
|
||||
public long getMobFlushCount() {
|
||||
return mobFlushCount;
|
||||
}
|
||||
|
||||
public void updateMobFlushedCellsCount(long count) {
|
||||
mobFlushedCellsCount += count;
|
||||
}
|
||||
|
||||
public long getMobFlushedCellsCount() {
|
||||
return mobFlushedCellsCount;
|
||||
}
|
||||
|
||||
public void updateMobFlushedCellsSize(long size) {
|
||||
mobFlushedCellsSize += size;
|
||||
}
|
||||
|
||||
public long getMobFlushedCellsSize() {
|
||||
return mobFlushedCellsSize;
|
||||
}
|
||||
|
||||
public void updateMobScanCellsCount(long count) {
|
||||
mobScanCellsCount += count;
|
||||
}
|
||||
|
||||
public long getMobScanCellsCount() {
|
||||
return mobScanCellsCount;
|
||||
}
|
||||
|
||||
public void updateMobScanCellsSize(long size) {
|
||||
mobScanCellsSize += size;
|
||||
}
|
||||
|
||||
public long getMobScanCellsSize() {
|
||||
return mobScanCellsSize;
|
||||
}
|
||||
}
|
|
@ -127,6 +127,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
|
|||
import org.apache.hadoop.hbase.ipc.CallerDisconnectedException;
|
||||
import org.apache.hadoop.hbase.ipc.RpcCallContext;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -3443,8 +3444,26 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir);
|
||||
|
||||
SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
|
||||
snapshotDir, desc, exnSnare);
|
||||
snapshotDir, desc, exnSnare);
|
||||
manifest.addRegion(this);
|
||||
|
||||
// The regionserver holding the first region of the table is responsible for taking the
|
||||
// manifest of the mob dir.
|
||||
if (!Bytes.equals(getRegionInfo().getStartKey(), HConstants.EMPTY_START_ROW))
|
||||
return;
|
||||
|
||||
// if any cf's have is mob enabled, add the "mob region" to the manifest.
|
||||
List<Store> stores = getStores();
|
||||
for (Store store : stores) {
|
||||
boolean hasMobStore = store.getFamily().isMobEnabled();
|
||||
if (hasMobStore) {
|
||||
// use the .mob as the start key and 0 as the regionid
|
||||
HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(this.getTableDesc().getTableName());
|
||||
mobRegionInfo.setOffline(true);
|
||||
manifest.addMobRegion(mobRegionInfo, this.getTableDesc().getColumnFamilies());
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -4875,6 +4894,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
|
||||
protected HStore instantiateHStore(final HColumnDescriptor family) throws IOException {
|
||||
if (family.isMobEnabled()) {
|
||||
if (HFile.getFormatVersion(this.conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
|
||||
throw new IOException("A minimum HFile version of "
|
||||
+ HFile.MIN_FORMAT_VERSION_WITH_TAGS
|
||||
+ " is required for MOB feature. Consider setting " + HFile.FORMAT_VERSION_KEY
|
||||
+ " accordingly.");
|
||||
}
|
||||
return new HMobStore(this, family, this.conf);
|
||||
}
|
||||
return new HStore(this, family, this.conf);
|
||||
}
|
||||
|
||||
|
|
|
@ -103,6 +103,7 @@ import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
|||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.mob.MobCacheConfig;
|
||||
import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
|
@ -387,6 +388,8 @@ public class HRegionServer extends HasThread implements
|
|||
|
||||
// Cache configuration and block cache reference
|
||||
protected CacheConfig cacheConfig;
|
||||
// Cache configuration for mob
|
||||
final MobCacheConfig mobCacheConfig;
|
||||
|
||||
/** The health check chore. */
|
||||
private HealthCheckChore healthCheckChore;
|
||||
|
@ -553,6 +556,8 @@ public class HRegionServer extends HasThread implements
|
|||
login(userProvider, hostName);
|
||||
|
||||
regionServerAccounting = new RegionServerAccounting();
|
||||
cacheConfig = new CacheConfig(conf);
|
||||
mobCacheConfig = new MobCacheConfig(conf);
|
||||
uncaughtExceptionHandler = new UncaughtExceptionHandler() {
|
||||
@Override
|
||||
public void uncaughtException(Thread t, Throwable e) {
|
||||
|
@ -984,6 +989,7 @@ public class HRegionServer extends HasThread implements
|
|||
if (cacheConfig != null && cacheConfig.isBlockCacheEnabled()) {
|
||||
cacheConfig.getBlockCache().shutdown();
|
||||
}
|
||||
mobCacheConfig.getMobFileCache().shutdown();
|
||||
|
||||
if (movedRegionsCleaner != null) {
|
||||
movedRegionsCleaner.stop("Region Server stopping");
|
||||
|
|
|
@ -127,11 +127,11 @@ public class HStore implements Store {
|
|||
|
||||
protected final MemStore memstore;
|
||||
// This stores directory in the filesystem.
|
||||
private final HRegion region;
|
||||
protected final HRegion region;
|
||||
private final HColumnDescriptor family;
|
||||
private final HRegionFileSystem fs;
|
||||
private Configuration conf;
|
||||
private final CacheConfig cacheConf;
|
||||
protected Configuration conf;
|
||||
protected CacheConfig cacheConf;
|
||||
private long lastCompactSize = 0;
|
||||
volatile boolean forceMajor = false;
|
||||
/* how many bytes to write between status checks */
|
||||
|
@ -160,12 +160,12 @@ public class HStore implements Store {
|
|||
private final Set<ChangedReadersObserver> changedReaderObservers =
|
||||
Collections.newSetFromMap(new ConcurrentHashMap<ChangedReadersObserver, Boolean>());
|
||||
|
||||
private final int blocksize;
|
||||
protected final int blocksize;
|
||||
private HFileDataBlockEncoder dataBlockEncoder;
|
||||
|
||||
/** Checksum configuration */
|
||||
private ChecksumType checksumType;
|
||||
private int bytesPerChecksum;
|
||||
protected ChecksumType checksumType;
|
||||
protected int bytesPerChecksum;
|
||||
|
||||
// Comparing KeyValues
|
||||
private final CellComparator comparator;
|
||||
|
@ -182,7 +182,7 @@ public class HStore implements Store {
|
|||
private long blockingFileCount;
|
||||
private int compactionCheckMultiplier;
|
||||
|
||||
private Encryption.Context cryptoContext = Encryption.Context.NONE;
|
||||
protected Encryption.Context cryptoContext = Encryption.Context.NONE;
|
||||
|
||||
private volatile long flushedCellsCount = 0;
|
||||
private volatile long compactedCellsCount = 0;
|
||||
|
@ -202,7 +202,6 @@ public class HStore implements Store {
|
|||
protected HStore(final HRegion region, final HColumnDescriptor family,
|
||||
final Configuration confParam) throws IOException {
|
||||
|
||||
HRegionInfo info = region.getRegionInfo();
|
||||
this.fs = region.getRegionFileSystem();
|
||||
|
||||
// Assemble the store's home directory and Ensure it exists.
|
||||
|
@ -239,7 +238,7 @@ public class HStore implements Store {
|
|||
this.offPeakHours = OffPeakHours.getInstance(conf);
|
||||
|
||||
// Setting up cache configuration for this family
|
||||
this.cacheConf = new CacheConfig(conf, family);
|
||||
createCacheConf(family);
|
||||
|
||||
this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
|
||||
|
||||
|
@ -258,7 +257,7 @@ public class HStore implements Store {
|
|||
"hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
|
||||
}
|
||||
|
||||
this.storeEngine = StoreEngine.create(this, this.conf, this.comparator);
|
||||
this.storeEngine = createStoreEngine(this, this.conf, this.comparator);
|
||||
this.storeEngine.getStoreFileManager().loadFiles(loadStoreFiles());
|
||||
|
||||
// Initialize checksum type from name. The names are CRC32, CRC32C, etc.
|
||||
|
@ -332,11 +331,32 @@ public class HStore implements Store {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the cache config.
|
||||
* @param family The current column family.
|
||||
*/
|
||||
protected void createCacheConf(final HColumnDescriptor family) {
|
||||
this.cacheConf = new CacheConfig(conf, family);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the store engine configured for the given Store.
|
||||
* @param store The store. An unfortunate dependency needed due to it
|
||||
* being passed to coprocessors via the compactor.
|
||||
* @param conf Store configuration.
|
||||
* @param kvComparator KVComparator for storeFileManager.
|
||||
* @return StoreEngine to use.
|
||||
*/
|
||||
protected StoreEngine<?, ?, ?, ?> createStoreEngine(Store store, Configuration conf,
|
||||
CellComparator kvComparator) throws IOException {
|
||||
return StoreEngine.create(store, conf, comparator);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param family
|
||||
* @return TTL in seconds of the specified family
|
||||
*/
|
||||
private static long determineTTLFromFamily(final HColumnDescriptor family) {
|
||||
public static long determineTTLFromFamily(final HColumnDescriptor family) {
|
||||
// HCD.getTimeToLive returns ttl in seconds. Convert to milliseconds.
|
||||
long ttl = family.getTimeToLive();
|
||||
if (ttl == HConstants.FOREVER) {
|
||||
|
@ -1837,17 +1857,23 @@ public class HStore implements Store {
|
|||
if (this.getCoprocessorHost() != null) {
|
||||
scanner = this.getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols);
|
||||
}
|
||||
if (scanner == null) {
|
||||
scanner = scan.isReversed() ? new ReversedStoreScanner(this,
|
||||
getScanInfo(), scan, targetCols, readPt) : new StoreScanner(this,
|
||||
getScanInfo(), scan, targetCols, readPt);
|
||||
}
|
||||
scanner = createScanner(scan, targetCols, readPt, scanner);
|
||||
return scanner;
|
||||
} finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
protected KeyValueScanner createScanner(Scan scan, final NavigableSet<byte[]> targetCols,
|
||||
long readPt, KeyValueScanner scanner) throws IOException {
|
||||
if (scanner == null) {
|
||||
scanner = scan.isReversed() ? new ReversedStoreScanner(this,
|
||||
getScanInfo(), scan, targetCols, readPt) : new StoreScanner(this,
|
||||
getScanInfo(), scan, targetCols, readPt);
|
||||
}
|
||||
return scanner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.getColumnFamilyName();
|
||||
|
|
|
@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache;
|
|||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheStats;
|
||||
import org.apache.hadoop.hbase.wal.BoundedRegionGroupingProvider;
|
||||
import org.apache.hadoop.hbase.mob.MobCacheConfig;
|
||||
import org.apache.hadoop.hbase.mob.MobFileCache;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
@ -55,6 +57,7 @@ class MetricsRegionServerWrapperImpl
|
|||
private final HRegionServer regionServer;
|
||||
|
||||
private BlockCache blockCache;
|
||||
private MobFileCache mobFileCache;
|
||||
|
||||
private volatile long numStores = 0;
|
||||
private volatile long numWALFiles = 0;
|
||||
|
@ -80,6 +83,20 @@ class MetricsRegionServerWrapperImpl
|
|||
private volatile long flushedCellsSize = 0;
|
||||
private volatile long compactedCellsSize = 0;
|
||||
private volatile long majorCompactedCellsSize = 0;
|
||||
private volatile long cellsCountCompactedToMob = 0;
|
||||
private volatile long cellsCountCompactedFromMob = 0;
|
||||
private volatile long cellsSizeCompactedToMob = 0;
|
||||
private volatile long cellsSizeCompactedFromMob = 0;
|
||||
private volatile long mobFlushCount = 0;
|
||||
private volatile long mobFlushedCellsCount = 0;
|
||||
private volatile long mobFlushedCellsSize = 0;
|
||||
private volatile long mobScanCellsCount = 0;
|
||||
private volatile long mobScanCellsSize = 0;
|
||||
private volatile long mobFileCacheAccessCount = 0;
|
||||
private volatile long mobFileCacheMissCount = 0;
|
||||
private volatile double mobFileCacheHitRatio = 0;
|
||||
private volatile long mobFileCacheEvictedCount = 0;
|
||||
private volatile long mobFileCacheCount = 0;
|
||||
private volatile long blockedRequestsCount = 0L;
|
||||
|
||||
private CacheStats cacheStats;
|
||||
|
@ -95,6 +112,7 @@ class MetricsRegionServerWrapperImpl
|
|||
public MetricsRegionServerWrapperImpl(final HRegionServer regionServer) {
|
||||
this.regionServer = regionServer;
|
||||
initBlockCache();
|
||||
initMobFileCache();
|
||||
|
||||
this.period =
|
||||
regionServer.conf.getLong(HConstants.REGIONSERVER_METRICS_PERIOD,
|
||||
|
@ -131,6 +149,16 @@ class MetricsRegionServerWrapperImpl
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes the mob file cache.
|
||||
*/
|
||||
private synchronized void initMobFileCache() {
|
||||
MobCacheConfig mobCacheConfig = this.regionServer.mobCacheConfig;
|
||||
if (mobCacheConfig != null && this.mobFileCache == null) {
|
||||
this.mobFileCache = mobCacheConfig.getMobFileCache();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getClusterId() {
|
||||
return regionServer.getClusterId();
|
||||
|
@ -421,6 +449,76 @@ class MetricsRegionServerWrapperImpl
|
|||
return majorCompactedCellsSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCellsCountCompactedFromMob() {
|
||||
return cellsCountCompactedFromMob;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCellsCountCompactedToMob() {
|
||||
return cellsCountCompactedToMob;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCellsSizeCompactedFromMob() {
|
||||
return cellsSizeCompactedFromMob;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCellsSizeCompactedToMob() {
|
||||
return cellsSizeCompactedToMob;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMobFlushCount() {
|
||||
return mobFlushCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMobFlushedCellsCount() {
|
||||
return mobFlushedCellsCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMobFlushedCellsSize() {
|
||||
return mobFlushedCellsSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMobScanCellsCount() {
|
||||
return mobScanCellsCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMobScanCellsSize() {
|
||||
return mobScanCellsSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMobFileCacheAccessCount() {
|
||||
return mobFileCacheAccessCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMobFileCacheMissCount() {
|
||||
return mobFileCacheMissCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMobFileCacheCount() {
|
||||
return mobFileCacheCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMobFileCacheEvictedCount() {
|
||||
return mobFileCacheEvictedCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMobFileCacheHitPercent() {
|
||||
return (int) (mobFileCacheHitRatio * 100);
|
||||
}
|
||||
|
||||
/**
|
||||
* This is the runnable that will be executed on the executor every PERIOD number of seconds
|
||||
* It will take metrics/numbers from all of the regions and use them to compute point in
|
||||
|
@ -434,6 +532,7 @@ class MetricsRegionServerWrapperImpl
|
|||
@Override
|
||||
synchronized public void run() {
|
||||
initBlockCache();
|
||||
initMobFileCache();
|
||||
cacheStats = blockCache.getStats();
|
||||
|
||||
HDFSBlocksDistribution hdfsBlocksDistribution =
|
||||
|
@ -441,12 +540,8 @@ class MetricsRegionServerWrapperImpl
|
|||
HDFSBlocksDistribution hdfsBlocksDistributionSecondaryRegions =
|
||||
new HDFSBlocksDistribution();
|
||||
|
||||
long tempNumStores = 0;
|
||||
long tempNumStoreFiles = 0;
|
||||
long tempMemstoreSize = 0;
|
||||
long tempStoreFileSize = 0;
|
||||
long tempReadRequestsCount = 0;
|
||||
long tempWriteRequestsCount = 0;
|
||||
long tempNumStores = 0, tempNumStoreFiles = 0, tempMemstoreSize = 0, tempStoreFileSize = 0;
|
||||
long tempReadRequestsCount = 0, tempWriteRequestsCount = 0;
|
||||
long tempCheckAndMutateChecksFailed = 0;
|
||||
long tempCheckAndMutateChecksPassed = 0;
|
||||
long tempStorefileIndexSize = 0;
|
||||
|
@ -462,6 +557,15 @@ class MetricsRegionServerWrapperImpl
|
|||
long tempFlushedCellsSize = 0;
|
||||
long tempCompactedCellsSize = 0;
|
||||
long tempMajorCompactedCellsSize = 0;
|
||||
long tempCellsCountCompactedToMob = 0;
|
||||
long tempCellsCountCompactedFromMob = 0;
|
||||
long tempCellsSizeCompactedToMob = 0;
|
||||
long tempCellsSizeCompactedFromMob = 0;
|
||||
long tempMobFlushCount = 0;
|
||||
long tempMobFlushedCellsCount = 0;
|
||||
long tempMobFlushedCellsSize = 0;
|
||||
long tempMobScanCellsCount = 0;
|
||||
long tempMobScanCellsSize = 0;
|
||||
long tempBlockedRequestsCount = 0L;
|
||||
|
||||
for (Region r : regionServer.getOnlineRegionsLocalContext()) {
|
||||
|
@ -487,6 +591,18 @@ class MetricsRegionServerWrapperImpl
|
|||
tempFlushedCellsSize += store.getFlushedCellsSize();
|
||||
tempCompactedCellsSize += store.getCompactedCellsSize();
|
||||
tempMajorCompactedCellsSize += store.getMajorCompactedCellsSize();
|
||||
if (store instanceof HMobStore) {
|
||||
HMobStore mobStore = (HMobStore) store;
|
||||
tempCellsCountCompactedToMob += mobStore.getCellsCountCompactedToMob();
|
||||
tempCellsCountCompactedFromMob += mobStore.getCellsCountCompactedFromMob();
|
||||
tempCellsSizeCompactedToMob += mobStore.getCellsSizeCompactedToMob();
|
||||
tempCellsSizeCompactedFromMob += mobStore.getCellsSizeCompactedFromMob();
|
||||
tempMobFlushCount += mobStore.getMobFlushCount();
|
||||
tempMobFlushedCellsCount += mobStore.getMobFlushedCellsCount();
|
||||
tempMobFlushedCellsSize += mobStore.getMobFlushedCellsSize();
|
||||
tempMobScanCellsCount += mobStore.getMobScanCellsCount();
|
||||
tempMobScanCellsSize += mobStore.getMobScanCellsSize();
|
||||
}
|
||||
}
|
||||
|
||||
HDFSBlocksDistribution distro = r.getHDFSBlocksDistribution();
|
||||
|
@ -495,7 +611,6 @@ class MetricsRegionServerWrapperImpl
|
|||
hdfsBlocksDistributionSecondaryRegions.add(distro);
|
||||
}
|
||||
}
|
||||
|
||||
float localityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(
|
||||
regionServer.getServerName().getHostname());
|
||||
tempPercentFileLocal = (int) (localityIndex * 100);
|
||||
|
@ -512,8 +627,6 @@ class MetricsRegionServerWrapperImpl
|
|||
if (lastRan == 0) {
|
||||
lastRan = currentTime - period;
|
||||
}
|
||||
|
||||
|
||||
//If we've time traveled keep the last requests per second.
|
||||
if ((currentTime - lastRan) > 0) {
|
||||
long currentRequestCount = getTotalRequestCount();
|
||||
|
@ -549,6 +662,20 @@ class MetricsRegionServerWrapperImpl
|
|||
flushedCellsSize = tempFlushedCellsSize;
|
||||
compactedCellsSize = tempCompactedCellsSize;
|
||||
majorCompactedCellsSize = tempMajorCompactedCellsSize;
|
||||
cellsCountCompactedToMob = tempCellsCountCompactedToMob;
|
||||
cellsCountCompactedFromMob = tempCellsCountCompactedFromMob;
|
||||
cellsSizeCompactedToMob = tempCellsSizeCompactedToMob;
|
||||
cellsSizeCompactedFromMob = tempCellsSizeCompactedFromMob;
|
||||
mobFlushCount = tempMobFlushCount;
|
||||
mobFlushedCellsCount = tempMobFlushedCellsCount;
|
||||
mobFlushedCellsSize = tempMobFlushedCellsSize;
|
||||
mobScanCellsCount = tempMobScanCellsCount;
|
||||
mobScanCellsSize = tempMobScanCellsSize;
|
||||
mobFileCacheAccessCount = mobFileCache.getAccessCount();
|
||||
mobFileCacheMissCount = mobFileCache.getMissCount();
|
||||
mobFileCacheHitRatio = mobFileCache.getHitRatio();
|
||||
mobFileCacheEvictedCount = mobFileCache.getEvictedFileCount();
|
||||
mobFileCacheCount = mobFileCache.getCacheSize();
|
||||
blockedRequestsCount = tempBlockedRequestsCount;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,66 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
||||
/**
|
||||
* Scanner scans the MOB Store. Coalesce KeyValue stream into List<KeyValue>
|
||||
* for a single row. It's only used in the compaction of mob-enabled columns.
|
||||
* It outputs the normal cells and delete markers when outputDeleteMarkers is set as true.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobCompactionStoreScanner extends StoreScanner {
|
||||
|
||||
/*
|
||||
* The delete markers are probably contained in the output of the scanner, for instance the
|
||||
* minor compaction. If outputDeleteMarkers is set as true, these delete markers could be
|
||||
* written to the del file, otherwise it's not allowed.
|
||||
*/
|
||||
protected boolean outputDeleteMarkers;
|
||||
|
||||
/**
|
||||
* Used for compactions.<p>
|
||||
*
|
||||
* Opens a scanner across specified StoreFiles.
|
||||
* @param store who we scan
|
||||
* @param scan the spec
|
||||
* @param scanners ancillary scanners
|
||||
* @param smallestReadPoint the readPoint that we should use for tracking
|
||||
* versions
|
||||
*/
|
||||
public MobCompactionStoreScanner(Store store, ScanInfo scanInfo, Scan scan,
|
||||
List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint,
|
||||
long earliestPutTs, boolean outputDeleteMarkers) throws IOException {
|
||||
super(store, scanInfo, scan, scanners, scanType, smallestReadPoint, earliestPutTs);
|
||||
this.outputDeleteMarkers = outputDeleteMarkers;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets whether the delete markers could be written to the del files.
|
||||
* @return True if the delete markers could be written del files, false if it's not allowed.
|
||||
*/
|
||||
public boolean isOutputDeleteMarkers() {
|
||||
return this.outputDeleteMarkers;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
|
||||
/**
|
||||
* A filter that returns the cells which have mob reference tags. It's a server-side filter.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class MobReferenceOnlyFilter extends FilterBase {
|
||||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell cell) {
|
||||
if (null != cell) {
|
||||
// If a cell with a mob reference tag, it's included.
|
||||
if (MobUtils.isMobReferenceCell(cell)) {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
}
|
||||
return ReturnCode.SKIP;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,85 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
|
||||
/**
|
||||
* Scanner scans both the memstore and the MOB Store. Coalesce KeyValue stream into List<KeyValue>
|
||||
* for a single row.
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobStoreScanner extends StoreScanner {
|
||||
|
||||
private boolean cacheMobBlocks = false;
|
||||
private boolean rawMobScan = false;
|
||||
private boolean readEmptyValueOnMobCellMiss = false;
|
||||
private final HMobStore mobStore;
|
||||
|
||||
public MobStoreScanner(Store store, ScanInfo scanInfo, Scan scan,
|
||||
final NavigableSet<byte[]> columns, long readPt) throws IOException {
|
||||
super(store, scanInfo, scan, columns, readPt);
|
||||
cacheMobBlocks = MobUtils.isCacheMobBlocks(scan);
|
||||
rawMobScan = MobUtils.isRawMobScan(scan);
|
||||
readEmptyValueOnMobCellMiss = MobUtils.isReadEmptyValueOnMobCellMiss(scan);
|
||||
if (!(store instanceof HMobStore)) {
|
||||
throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
|
||||
}
|
||||
mobStore = (HMobStore) store;
|
||||
}
|
||||
|
||||
/**
|
||||
* Firstly reads the cells from the HBase. If the cell are a reference cell (which has the
|
||||
* reference tag), the scanner need seek this cell from the mob file, and use the cell found
|
||||
* from the mob file as the result.
|
||||
*/
|
||||
@Override
|
||||
public boolean next(List<Cell> outResult, ScannerContext ctx) throws IOException {
|
||||
boolean result = super.next(outResult, ctx);
|
||||
if (!rawMobScan) {
|
||||
// retrieve the mob data
|
||||
if (outResult.isEmpty()) {
|
||||
return result;
|
||||
}
|
||||
long mobKVCount = 0;
|
||||
long mobKVSize = 0;
|
||||
for (int i = 0; i < outResult.size(); i++) {
|
||||
Cell cell = outResult.get(i);
|
||||
if (MobUtils.isMobReferenceCell(cell)) {
|
||||
Cell mobCell = mobStore
|
||||
.resolve(cell, cacheMobBlocks, readPt, readEmptyValueOnMobCellMiss);
|
||||
mobKVCount++;
|
||||
mobKVSize += mobCell.getValueLength();
|
||||
outResult.set(i, mobCell);
|
||||
}
|
||||
}
|
||||
mobStore.updateMobScanCellsCount(mobKVCount);
|
||||
mobStore.updateMobScanCellsSize(mobKVSize);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,85 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
|
||||
/**
|
||||
* ReversedMobStoreScanner extends from ReversedStoreScanner, and is used to support
|
||||
* reversed scanning in both the memstore and the MOB store.
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ReversedMobStoreScanner extends ReversedStoreScanner {
|
||||
|
||||
private boolean cacheMobBlocks = false;
|
||||
private boolean rawMobScan = false;
|
||||
private boolean readEmptyValueOnMobCellMiss = false;
|
||||
protected final HMobStore mobStore;
|
||||
|
||||
ReversedMobStoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
|
||||
long readPt) throws IOException {
|
||||
super(store, scanInfo, scan, columns, readPt);
|
||||
cacheMobBlocks = MobUtils.isCacheMobBlocks(scan);
|
||||
rawMobScan = MobUtils.isRawMobScan(scan);
|
||||
readEmptyValueOnMobCellMiss = MobUtils.isReadEmptyValueOnMobCellMiss(scan);
|
||||
if (!(store instanceof HMobStore)) {
|
||||
throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
|
||||
}
|
||||
mobStore = (HMobStore) store;
|
||||
}
|
||||
|
||||
/**
|
||||
* Firstly reads the cells from the HBase. If the cell is a reference cell (which has the
|
||||
* reference tag), the scanner need seek this cell from the mob file, and use the cell found
|
||||
* from the mob file as the result.
|
||||
*/
|
||||
@Override
|
||||
public boolean next(List<Cell> outResult, ScannerContext ctx) throws IOException {
|
||||
boolean result = super.next(outResult, ctx);
|
||||
if (!rawMobScan) {
|
||||
// retrieve the mob data
|
||||
if (outResult.isEmpty()) {
|
||||
return result;
|
||||
}
|
||||
long mobKVCount = 0;
|
||||
long mobKVSize = 0;
|
||||
for (int i = 0; i < outResult.size(); i++) {
|
||||
Cell cell = outResult.get(i);
|
||||
if (MobUtils.isMobReferenceCell(cell)) {
|
||||
Cell mobCell = mobStore
|
||||
.resolve(cell, cacheMobBlocks, readPt, readEmptyValueOnMobCellMiss);
|
||||
mobKVCount++;
|
||||
mobKVSize += mobCell.getValueLength();
|
||||
outResult.set(i, mobCell);
|
||||
}
|
||||
}
|
||||
mobStore.updateMobScanCellsCount(mobKVCount);
|
||||
mobStore.updateMobScanCellsSize(mobKVSize);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -112,6 +112,9 @@ public class StoreFile {
|
|||
/** Key for timestamp of earliest-put in metadata*/
|
||||
public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
|
||||
|
||||
/** Key for the number of mob cells in metadata*/
|
||||
public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
|
||||
|
||||
private final StoreFileInfo fileInfo;
|
||||
private final FileSystem fs;
|
||||
|
||||
|
@ -162,6 +165,13 @@ public class StoreFile {
|
|||
*/
|
||||
private final BloomType cfBloomType;
|
||||
|
||||
/**
|
||||
* Key for skipping resetting sequence id in metadata.
|
||||
* For bulk loaded hfiles, the scanner resets the cell seqId with the latest one,
|
||||
* if this metadata is set as true, the reset is skipped.
|
||||
*/
|
||||
public static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
|
||||
|
||||
/**
|
||||
* Constructor, loads a reader and it's indices, etc. May allocate a
|
||||
* substantial amount of ram depending on the underlying files (10-20MB?).
|
||||
|
@ -405,6 +415,12 @@ public class StoreFile {
|
|||
this.sequenceid += 1;
|
||||
}
|
||||
}
|
||||
// SKIP_RESET_SEQ_ID only works in bulk loaded file.
|
||||
// In mob compaction, the hfile where the cells contain the path of a new mob file is bulk
|
||||
// loaded to hbase, these cells have the same seqIds with the old ones. We do not want
|
||||
// to reset new seqIds for them since this might make a mess of the visibility of cells that
|
||||
// have the same row key but different seqIds.
|
||||
this.reader.setSkipResetSeqId(isSkipResetSeqId(metadataMap.get(SKIP_RESET_SEQ_ID)));
|
||||
this.reader.setBulkLoaded(true);
|
||||
}
|
||||
this.reader.setSequenceID(this.sequenceid);
|
||||
|
@ -534,6 +550,18 @@ public class StoreFile {
|
|||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets whether to skip resetting the sequence id for cells.
|
||||
* @param skipResetSeqId The byte array of boolean.
|
||||
* @return Whether to skip resetting the sequence id.
|
||||
*/
|
||||
private boolean isSkipResetSeqId(byte[] skipResetSeqId) {
|
||||
if (skipResetSeqId != null && skipResetSeqId.length == 1) {
|
||||
return Bytes.toBoolean(skipResetSeqId);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static class WriterBuilder {
|
||||
private final Configuration conf;
|
||||
private final CacheConfig cacheConf;
|
||||
|
@ -798,6 +826,22 @@ public class StoreFile {
|
|||
appendTrackedTimestampsToMetadata();
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes meta data.
|
||||
* Call before {@link #close()} since its written as meta data to this file.
|
||||
* @param maxSequenceId Maximum sequence id.
|
||||
* @param majorCompaction True if this file is product of a major compaction
|
||||
* @param mobCellsCount The number of mob cells.
|
||||
* @throws IOException problem writing to FS
|
||||
*/
|
||||
public void appendMetadata(final long maxSequenceId, final boolean majorCompaction,
|
||||
final long mobCellsCount) throws IOException {
|
||||
writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
|
||||
writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction));
|
||||
writer.appendFileInfo(MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount));
|
||||
appendTrackedTimestampsToMetadata();
|
||||
}
|
||||
|
||||
/**
|
||||
* Add TimestampRange and earliest put timestamp to Metadata
|
||||
*/
|
||||
|
@ -951,7 +995,7 @@ public class StoreFile {
|
|||
return this.writer.getPath();
|
||||
}
|
||||
|
||||
boolean hasGeneralBloom() {
|
||||
public boolean hasGeneralBloom() {
|
||||
return this.generalBloomFilterWriter != null;
|
||||
}
|
||||
|
||||
|
@ -1048,6 +1092,7 @@ public class StoreFile {
|
|||
private long deleteFamilyCnt = -1;
|
||||
private boolean bulkLoadResult = false;
|
||||
private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
|
||||
private boolean skipResetSeqId = true;
|
||||
|
||||
public Reader(FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf)
|
||||
throws IOException {
|
||||
|
@ -1570,6 +1615,14 @@ public class StoreFile {
|
|||
public long getMaxTimestamp() {
|
||||
return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.getMaximumTimestamp();
|
||||
}
|
||||
|
||||
boolean isSkipResetSeqId() {
|
||||
return skipResetSeqId;
|
||||
}
|
||||
|
||||
void setSkipResetSeqId(boolean skipResetSeqId) {
|
||||
this.skipResetSeqId = skipResetSeqId;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -49,13 +49,24 @@ public class StoreFileInfo {
|
|||
/**
|
||||
* A non-capture group, for hfiles, so that this can be embedded.
|
||||
* HFiles are uuid ([0-9a-z]+). Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix.
|
||||
* The mob del file has (_del) as suffix.
|
||||
*/
|
||||
public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:_SeqId_[0-9]+_)?";
|
||||
public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:(?:_SeqId_[0-9]+_)|(?:_del))?";
|
||||
|
||||
/** Regex that will work for hfiles */
|
||||
private static final Pattern HFILE_NAME_PATTERN =
|
||||
Pattern.compile("^(" + HFILE_NAME_REGEX + ")");
|
||||
|
||||
/**
|
||||
* A non-capture group, for del files, so that this can be embedded.
|
||||
* A del file has (_del) as suffix.
|
||||
*/
|
||||
public static final String DELFILE_NAME_REGEX = "[0-9a-f]+(?:_del)";
|
||||
|
||||
/** Regex that will work for del files */
|
||||
private static final Pattern DELFILE_NAME_PATTERN =
|
||||
Pattern.compile("^(" + DELFILE_NAME_REGEX + ")");
|
||||
|
||||
/**
|
||||
* Regex that will work for straight reference names (<hfile>.<parentEncRegion>)
|
||||
* and hfilelink reference names (<table>=<region>-<hfile>.<parentEncRegion>)
|
||||
|
@ -371,6 +382,23 @@ public class StoreFileInfo {
|
|||
return m.matches() && m.groupCount() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param path Path to check.
|
||||
* @return True if the path has format of a del file.
|
||||
*/
|
||||
public static boolean isDelFile(final Path path) {
|
||||
return isDelFile(path.getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param fileName Sting version of path to validate.
|
||||
* @return True if the file name has format of a del file.
|
||||
*/
|
||||
public static boolean isDelFile(final String fileName) {
|
||||
Matcher m = DELFILE_NAME_PATTERN.matcher(fileName);
|
||||
return m.matches() && m.groupCount() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param path Path to check.
|
||||
* @return True if the path has format of a HStoreFile reference.
|
||||
|
|
|
@ -204,7 +204,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
|
||||
protected void setCurrentCell(Cell newVal) throws IOException {
|
||||
this.cur = newVal;
|
||||
if (this.cur != null && this.reader.isBulkLoaded()) {
|
||||
if (this.cur != null && this.reader.isBulkLoaded() && !this.reader.isSkipResetSeqId()) {
|
||||
CellUtil.setSequenceId(cur, this.reader.getSequenceID());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -125,7 +125,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
private boolean scanUsePread = false;
|
||||
protected ReentrantLock lock = new ReentrantLock();
|
||||
|
||||
private final long readPt;
|
||||
protected final long readPt;
|
||||
|
||||
// used by the injection framework to test race between StoreScanner construction and compaction
|
||||
enum StoreScannerCompactionRace {
|
||||
|
@ -307,8 +307,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
// 0 is passed as readpoint because the test bypasses Store
|
||||
0);
|
||||
}
|
||||
|
||||
private StoreScanner(final Scan scan, ScanInfo scanInfo,
|
||||
|
||||
public StoreScanner(final Scan scan, ScanInfo scanInfo,
|
||||
ScanType scanType, final NavigableSet<byte[]> columns,
|
||||
final List<KeyValueScanner> scanners, long earliestPutTs, long readPt)
|
||||
throws IOException {
|
||||
|
|
|
@ -59,6 +59,8 @@ public class CompactionRequest implements Comparable<CompactionRequest> {
|
|||
private String storeName = "";
|
||||
private long totalSize = -1L;
|
||||
|
||||
private Boolean retainDeleteMarkers = null;
|
||||
|
||||
/**
|
||||
* This ctor should be used by coprocessors that want to subclass CompactionRequest.
|
||||
*/
|
||||
|
@ -200,6 +202,23 @@ public class CompactionRequest implements Comparable<CompactionRequest> {
|
|||
: (isMajor ? DisplayCompactionType.MAJOR : DisplayCompactionType.ALL_FILES);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forcefully setting that this compaction has to retain the delete markers in the new compacted
|
||||
* file, whatever be the type of the compaction.<br>
|
||||
* Note : By default HBase drops delete markers when the compaction is on all files.
|
||||
*/
|
||||
public void forceRetainDeleteMarkers() {
|
||||
this.retainDeleteMarkers = Boolean.TRUE;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Whether the compaction has to retain the delete markers or not.
|
||||
*/
|
||||
public boolean isRetainDeleteMarkers() {
|
||||
return (this.retainDeleteMarkers != null) ? this.retainDeleteMarkers.booleanValue()
|
||||
: !isAllFiles();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
String fsList = Joiner.on(", ").join(
|
||||
|
|
|
@ -45,7 +45,9 @@ import org.apache.hadoop.hbase.regionserver.Store;
|
|||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
|
||||
|
||||
|
@ -60,7 +62,7 @@ public abstract class Compactor {
|
|||
protected Configuration conf;
|
||||
protected Store store;
|
||||
|
||||
private int compactionKVMax;
|
||||
protected int compactionKVMax;
|
||||
protected Compression.Algorithm compactionCompression;
|
||||
|
||||
/** specify how many days to keep MVCC values during major compaction **/
|
||||
|
@ -92,6 +94,8 @@ public abstract class Compactor {
|
|||
public long maxKeyCount = 0;
|
||||
/** Earliest put timestamp if major compaction */
|
||||
public long earliestPutTs = HConstants.LATEST_TIMESTAMP;
|
||||
/** Latest put timestamp */
|
||||
public long latestPutTs = HConstants.LATEST_TIMESTAMP;
|
||||
/** The last key in the files we're compacting. */
|
||||
public long maxSeqId = 0;
|
||||
/** Latest memstore read point found in any of the involved files */
|
||||
|
@ -166,6 +170,14 @@ public abstract class Compactor {
|
|||
fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
|
||||
}
|
||||
}
|
||||
tmp = fileInfo.get(StoreFile.TIMERANGE_KEY);
|
||||
TimeRangeTracker trt = new TimeRangeTracker();
|
||||
if (tmp == null) {
|
||||
fd.latestPutTs = HConstants.LATEST_TIMESTAMP;
|
||||
} else {
|
||||
Writables.copyWritable(tmp, trt);
|
||||
fd.latestPutTs = trt.getMaximumTimestamp();
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Compacting " + file +
|
||||
", keycount=" + keyCount +
|
||||
|
@ -240,17 +252,20 @@ public abstract class Compactor {
|
|||
return store.getRegionInfo().getRegionNameAsString() + "#"
|
||||
+ store.getFamily().getNameAsString() + "#" + counter;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs the compaction.
|
||||
* @param fd FileDetails of cell sink writer
|
||||
* @param scanner Where to read from.
|
||||
* @param writer Where to write to.
|
||||
* @param smallestReadPoint Smallest read point.
|
||||
* @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= smallestReadPoint
|
||||
* @param major Is a major compaction.
|
||||
* @return Whether compaction ended; false if it was interrupted for some reason.
|
||||
*/
|
||||
protected boolean performCompaction(InternalScanner scanner, CellSink writer,
|
||||
protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
|
||||
long smallestReadPoint, boolean cleanSeqId,
|
||||
CompactionThroughputController throughputController) throws IOException {
|
||||
CompactionThroughputController throughputController, boolean major) throws IOException {
|
||||
long bytesWritten = 0;
|
||||
long bytesWrittenProgress = 0;
|
||||
// Since scanner.next() can return 'false' but still be delivering data,
|
||||
|
@ -360,4 +375,17 @@ public abstract class Compactor {
|
|||
return new StoreScanner(store, store.getScanInfo(), scan, scanners, smallestReadPoint,
|
||||
earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends the metadata and closes the writer.
|
||||
* @param writer The current store writer.
|
||||
* @param fd The file details.
|
||||
* @param isMajor Is a major compaction.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void appendMetadataAndCloseWriter(StoreFile.Writer writer, FileDetails fd,
|
||||
boolean isMajor) throws IOException {
|
||||
writer.appendMetadata(fd.maxSeqId, isMajor);
|
||||
writer.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -81,8 +81,8 @@ public class DefaultCompactor extends Compactor {
|
|||
InternalScanner scanner = null;
|
||||
try {
|
||||
/* Include deletes, unless we are doing a compaction of all files */
|
||||
ScanType scanType =
|
||||
request.isAllFiles() ? ScanType.COMPACT_DROP_DELETES : ScanType.COMPACT_RETAIN_DELETES;
|
||||
ScanType scanType = request.isRetainDeleteMarkers() ? ScanType.COMPACT_RETAIN_DELETES
|
||||
: ScanType.COMPACT_DROP_DELETES;
|
||||
scanner = preCreateCoprocScanner(request, scanType, fd.earliestPutTs, scanners);
|
||||
if (scanner == null) {
|
||||
scanner = createScanner(store, scanners, scanType, smallestReadPoint, fd.earliestPutTs);
|
||||
|
@ -99,12 +99,9 @@ public class DefaultCompactor extends Compactor {
|
|||
cleanSeqId = true;
|
||||
}
|
||||
|
||||
// When all MVCC readpoints are 0, don't write them.
|
||||
// See HBASE-8166, HBASE-12600, and HBASE-13389.
|
||||
writer = store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression, true,
|
||||
fd.maxMVCCReadpoint > 0, fd.maxTagsLength > 0);
|
||||
boolean finished =
|
||||
performCompaction(scanner, writer, smallestReadPoint, cleanSeqId, throughputController);
|
||||
writer = createTmpWriter(fd, smallestReadPoint);
|
||||
boolean finished = performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId,
|
||||
throughputController, request.isAllFiles());
|
||||
if (!finished) {
|
||||
writer.close();
|
||||
store.getFileSystem().delete(writer.getPath(), false);
|
||||
|
@ -147,6 +144,24 @@ public class DefaultCompactor extends Compactor {
|
|||
return newFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a writer for a new file in a temporary directory.
|
||||
* @param fd The file details.
|
||||
* @param smallestReadPoint The smallest mvcc readPoint across all the scanners in this region.
|
||||
* @return Writer for a new StoreFile in the tmp dir.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected StoreFile.Writer createTmpWriter(FileDetails fd, long smallestReadPoint)
|
||||
throws IOException {
|
||||
// When all MVCC readpoints are 0, don't write them.
|
||||
// See HBASE-8166, HBASE-12600, and HBASE-13389.
|
||||
|
||||
// make this writer with tags always because of possible new cells with tags.
|
||||
return store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression,
|
||||
true, fd.maxMVCCReadpoint >= 0, fd.maxTagsLength >0);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Compact a list of files for testing. Creates a fake {@link CompactionRequest} to pass to
|
||||
* {@link #compact(CompactionRequest, CompactionThroughputController)};
|
||||
|
|
|
@ -133,7 +133,8 @@ public class StripeCompactor extends Compactor {
|
|||
StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner)scanner : null;
|
||||
mw.init(storeScanner, factory, store.getComparator());
|
||||
finished =
|
||||
performCompaction(scanner, mw, smallestReadPoint, cleanSeqId, throughputController);
|
||||
performCompaction(fd, scanner, mw, smallestReadPoint, cleanSeqId, throughputController,
|
||||
request.isMajor());
|
||||
if (!finished) {
|
||||
throw new InterruptedIOException( "Aborting compaction of store " + store +
|
||||
" in region " + store.getRegionInfo().getRegionNameAsString() +
|
||||
|
|
|
@ -53,10 +53,12 @@ import org.apache.hadoop.hbase.io.FileLink;
|
|||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.io.WALLink;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
@ -108,8 +110,6 @@ public class ExportSnapshot extends Configured implements Tool {
|
|||
static final String CONF_TEST_FAILURE = "test.snapshot.export.failure";
|
||||
static final String CONF_TEST_RETRY = "test.snapshot.export.failure.retry";
|
||||
|
||||
private static final String INPUT_FOLDER_PREFIX = "export-files.";
|
||||
|
||||
// Export Map-Reduce Counters, to keep track of the progress
|
||||
public enum Counter {
|
||||
MISSING_FILES, FILES_COPIED, FILES_SKIPPED, COPY_FAILED,
|
||||
|
@ -409,7 +409,7 @@ public class ExportSnapshot extends Configured implements Tool {
|
|||
switch (fileInfo.getType()) {
|
||||
case HFILE:
|
||||
Path inputPath = new Path(fileInfo.getHfile());
|
||||
link = HFileLink.buildFromHFileLinkPattern(conf, inputPath);
|
||||
link = getFileLink(inputPath, conf);
|
||||
break;
|
||||
case WAL:
|
||||
String serverName = fileInfo.getWalServer();
|
||||
|
@ -435,7 +435,7 @@ public class ExportSnapshot extends Configured implements Tool {
|
|||
switch (fileInfo.getType()) {
|
||||
case HFILE:
|
||||
Path inputPath = new Path(fileInfo.getHfile());
|
||||
link = HFileLink.buildFromHFileLinkPattern(conf, inputPath);
|
||||
link = getFileLink(inputPath, conf);
|
||||
break;
|
||||
case WAL:
|
||||
link = new WALLink(inputRoot, fileInfo.getWalServer(), fileInfo.getWalName());
|
||||
|
@ -454,6 +454,16 @@ public class ExportSnapshot extends Configured implements Tool {
|
|||
}
|
||||
}
|
||||
|
||||
private FileLink getFileLink(Path path, Configuration conf) throws IOException{
|
||||
String regionName = HFileLink.getReferencedRegionName(path.getName());
|
||||
TableName tableName = HFileLink.getReferencedTableName(path.getName());
|
||||
if(MobUtils.getMobRegionInfo(tableName).getEncodedName().equals(regionName)) {
|
||||
return HFileLink.buildFromHFileLinkPattern(MobUtils.getQualifiedMobRootDir(conf),
|
||||
HFileArchiveUtil.getArchivePath(conf), path);
|
||||
}
|
||||
return HFileLink.buildFromHFileLinkPattern(inputRoot, inputArchive, path);
|
||||
}
|
||||
|
||||
private FileChecksum getFileChecksum(final FileSystem fs, final Path path) {
|
||||
try {
|
||||
return fs.getFileChecksum(path);
|
||||
|
@ -507,9 +517,8 @@ public class ExportSnapshot extends Configured implements Tool {
|
|||
@Override
|
||||
public void storeFile(final HRegionInfo regionInfo, final String family,
|
||||
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
|
||||
if (storeFile.hasReference()) {
|
||||
// copied as part of the manifest
|
||||
} else {
|
||||
// for storeFile.hasReference() case, copied as part of the manifest
|
||||
if (!storeFile.hasReference()) {
|
||||
String region = regionInfo.getEncodedName();
|
||||
String hfile = storeFile.getName();
|
||||
Path path = HFileLink.createPath(table, region, family, hfile);
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.Connection;
|
|||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.io.Reference;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
|
@ -175,6 +176,8 @@ public class RestoreSnapshotHelper {
|
|||
// this instance, by removing the regions already present in the restore dir.
|
||||
Set<String> regionNames = new HashSet<String>(regionManifests.keySet());
|
||||
|
||||
HRegionInfo mobRegion = MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor()
|
||||
.getTableName());
|
||||
// Identify which region are still available and which not.
|
||||
// NOTE: we rely upon the region name as: "table name, start key, end key"
|
||||
List<HRegionInfo> tableRegions = getTableRegions();
|
||||
|
@ -195,6 +198,13 @@ public class RestoreSnapshotHelper {
|
|||
// Restore regions using the snapshot data
|
||||
monitor.rethrowException();
|
||||
status.setStatus("Restoring table regions...");
|
||||
if (regionNames.contains(mobRegion.getEncodedName())) {
|
||||
// restore the mob region in case
|
||||
List<HRegionInfo> mobRegions = new ArrayList<HRegionInfo>(1);
|
||||
mobRegions.add(mobRegion);
|
||||
restoreHdfsMobRegions(exec, regionManifests, mobRegions);
|
||||
regionNames.remove(mobRegion.getEncodedName());
|
||||
}
|
||||
restoreHdfsRegions(exec, regionManifests, metaChanges.getRegionsToRestore());
|
||||
status.setStatus("Finished restoring all table regions.");
|
||||
|
||||
|
@ -210,6 +220,11 @@ public class RestoreSnapshotHelper {
|
|||
List<HRegionInfo> regionsToAdd = new ArrayList<HRegionInfo>(regionNames.size());
|
||||
|
||||
monitor.rethrowException();
|
||||
// add the mob region
|
||||
if (regionNames.contains(mobRegion.getEncodedName())) {
|
||||
cloneHdfsMobRegion(regionManifests, mobRegion);
|
||||
regionNames.remove(mobRegion.getEncodedName());
|
||||
}
|
||||
for (String regionName: regionNames) {
|
||||
LOG.info("region to add: " + regionName);
|
||||
regionsToAdd.add(HRegionInfo.convert(regionManifests.get(regionName).getRegionInfo()));
|
||||
|
@ -379,6 +394,21 @@ public class RestoreSnapshotHelper {
|
|||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Restore specified mob regions by restoring content to the snapshot state.
|
||||
*/
|
||||
private void restoreHdfsMobRegions(final ThreadPoolExecutor exec,
|
||||
final Map<String, SnapshotRegionManifest> regionManifests,
|
||||
final List<HRegionInfo> regions) throws IOException {
|
||||
if (regions == null || regions.size() == 0) return;
|
||||
ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
|
||||
@Override
|
||||
public void editRegion(final HRegionInfo hri) throws IOException {
|
||||
restoreMobRegion(hri, regionManifests.get(hri.getEncodedName()));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Map<String, List<SnapshotRegionManifest.StoreFile>> getRegionHFileReferences(
|
||||
final SnapshotRegionManifest manifest) {
|
||||
Map<String, List<SnapshotRegionManifest.StoreFile>> familyMap =
|
||||
|
@ -396,10 +426,31 @@ public class RestoreSnapshotHelper {
|
|||
*/
|
||||
private void restoreRegion(final HRegionInfo regionInfo,
|
||||
final SnapshotRegionManifest regionManifest) throws IOException {
|
||||
restoreRegion(regionInfo, regionManifest, new Path(tableDir, regionInfo.getEncodedName()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Restore mob region by removing files not in the snapshot
|
||||
* and adding the missing ones from the snapshot.
|
||||
*/
|
||||
private void restoreMobRegion(final HRegionInfo regionInfo,
|
||||
final SnapshotRegionManifest regionManifest) throws IOException {
|
||||
if (regionManifest == null) {
|
||||
return;
|
||||
}
|
||||
restoreRegion(regionInfo, regionManifest,
|
||||
MobUtils.getMobRegionPath(conf, tableDesc.getTableName()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Restore region by removing files not in the snapshot
|
||||
* and adding the missing ones from the snapshot.
|
||||
*/
|
||||
private void restoreRegion(final HRegionInfo regionInfo,
|
||||
final SnapshotRegionManifest regionManifest, Path regionDir) throws IOException {
|
||||
Map<String, List<SnapshotRegionManifest.StoreFile>> snapshotFiles =
|
||||
getRegionHFileReferences(regionManifest);
|
||||
|
||||
Path regionDir = new Path(tableDir, regionInfo.getEncodedName());
|
||||
String tableName = tableDesc.getTableName().getNameAsString();
|
||||
|
||||
// Restore families present in the table
|
||||
|
@ -518,6 +569,40 @@ public class RestoreSnapshotHelper {
|
|||
return clonedRegionsInfo;
|
||||
}
|
||||
|
||||
/**
|
||||
* Clone the mob region. For the region create a new region
|
||||
* and create a HFileLink for each hfile.
|
||||
*/
|
||||
private void cloneHdfsMobRegion(final Map<String, SnapshotRegionManifest> regionManifests,
|
||||
final HRegionInfo region) throws IOException {
|
||||
// clone region info (change embedded tableName with the new one)
|
||||
Path clonedRegionPath = MobUtils.getMobRegionPath(conf, tableDesc.getTableName());
|
||||
cloneRegion(clonedRegionPath, region, regionManifests.get(region.getEncodedName()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Clone region directory content from the snapshot info.
|
||||
*
|
||||
* Each region is encoded with the table name, so the cloned region will have
|
||||
* a different region name.
|
||||
*
|
||||
* Instead of copying the hfiles a HFileLink is created.
|
||||
*
|
||||
* @param regionDir {@link Path} cloned dir
|
||||
* @param snapshotRegionInfo
|
||||
*/
|
||||
private void cloneRegion(final Path regionDir, final HRegionInfo snapshotRegionInfo,
|
||||
final SnapshotRegionManifest manifest) throws IOException {
|
||||
final String tableName = tableDesc.getTableName().getNameAsString();
|
||||
for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) {
|
||||
Path familyDir = new Path(regionDir, familyFiles.getFamilyName().toStringUtf8());
|
||||
for (SnapshotRegionManifest.StoreFile storeFile: familyFiles.getStoreFilesList()) {
|
||||
LOG.info("Adding HFileLink " + storeFile.getName() + " to table=" + tableName);
|
||||
restoreStoreFile(familyDir, snapshotRegionInfo, storeFile);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Clone region directory content from the snapshot info.
|
||||
*
|
||||
|
@ -531,15 +616,8 @@ public class RestoreSnapshotHelper {
|
|||
*/
|
||||
private void cloneRegion(final HRegion region, final HRegionInfo snapshotRegionInfo,
|
||||
final SnapshotRegionManifest manifest) throws IOException {
|
||||
final Path regionDir = new Path(tableDir, region.getRegionInfo().getEncodedName());
|
||||
final String tableName = tableDesc.getTableName().getNameAsString();
|
||||
for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) {
|
||||
Path familyDir = new Path(regionDir, familyFiles.getFamilyName().toStringUtf8());
|
||||
for (SnapshotRegionManifest.StoreFile storeFile: familyFiles.getStoreFilesList()) {
|
||||
LOG.info("Adding HFileLink " + storeFile.getName() + " to table=" + tableName);
|
||||
restoreStoreFile(familyDir, snapshotRegionInfo, storeFile);
|
||||
}
|
||||
}
|
||||
cloneRegion(new Path(tableDir, region.getRegionInfo().getEncodedName()), snapshotRegionInfo,
|
||||
manifest);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -712,8 +790,8 @@ public class RestoreSnapshotHelper {
|
|||
Path restoreDir, String snapshotName) throws IOException {
|
||||
// ensure that restore dir is not under root dir
|
||||
if (!restoreDir.getFileSystem(conf).getUri().equals(rootDir.getFileSystem(conf).getUri())) {
|
||||
throw new IllegalArgumentException("Filesystems for restore directory and HBase root directory " +
|
||||
"should be the same");
|
||||
throw new IllegalArgumentException("Filesystems for restore directory and HBase root " +
|
||||
"directory should be the same");
|
||||
}
|
||||
if (restoreDir.toUri().getPath().startsWith(rootDir.toUri().getPath())) {
|
||||
throw new IllegalArgumentException("Restore directory cannot be a sub directory of HBase " +
|
||||
|
|
|
@ -114,14 +114,16 @@ public final class SnapshotInfo extends Configured implements Tool {
|
|||
}
|
||||
}
|
||||
|
||||
private AtomicInteger hfileArchiveCount = new AtomicInteger();
|
||||
private AtomicInteger hfilesArchiveCount = new AtomicInteger();
|
||||
private AtomicInteger hfilesCorrupted = new AtomicInteger();
|
||||
private AtomicInteger hfilesMissing = new AtomicInteger();
|
||||
private AtomicInteger hfilesCount = new AtomicInteger();
|
||||
private AtomicInteger hfilesMobCount = new AtomicInteger();
|
||||
private AtomicInteger logsMissing = new AtomicInteger();
|
||||
private AtomicInteger logsCount = new AtomicInteger();
|
||||
private AtomicLong hfileArchiveSize = new AtomicLong();
|
||||
private AtomicLong hfileSize = new AtomicLong();
|
||||
private AtomicLong hfilesArchiveSize = new AtomicLong();
|
||||
private AtomicLong hfilesSize = new AtomicLong();
|
||||
private AtomicLong hfilesMobSize = new AtomicLong();
|
||||
private AtomicLong logSize = new AtomicLong();
|
||||
|
||||
private final SnapshotDescription snapshot;
|
||||
|
@ -151,14 +153,17 @@ public final class SnapshotInfo extends Configured implements Tool {
|
|||
|
||||
/** @return the number of available store files */
|
||||
public int getStoreFilesCount() {
|
||||
return hfilesCount.get() + hfileArchiveCount.get();
|
||||
return hfilesCount.get() + hfilesArchiveCount.get() + hfilesMobCount.get();
|
||||
}
|
||||
|
||||
/** @return the number of available store files in the archive */
|
||||
public int getArchivedStoreFilesCount() {
|
||||
return hfileArchiveCount.get();
|
||||
return hfilesArchiveCount.get();
|
||||
}
|
||||
|
||||
/** @return the number of available store files in the mob dir */
|
||||
public int getMobStoreFilesCount() { return hfilesMobCount.get(); }
|
||||
|
||||
/** @return the number of available log files */
|
||||
public int getLogsCount() {
|
||||
return logsCount.get();
|
||||
|
@ -181,22 +186,30 @@ public final class SnapshotInfo extends Configured implements Tool {
|
|||
|
||||
/** @return the total size of the store files referenced by the snapshot */
|
||||
public long getStoreFilesSize() {
|
||||
return hfileSize.get() + hfileArchiveSize.get();
|
||||
return hfilesSize.get() + hfilesArchiveSize.get() + hfilesMobSize.get();
|
||||
}
|
||||
|
||||
/** @return the total size of the store files shared */
|
||||
public long getSharedStoreFilesSize() {
|
||||
return hfileSize.get();
|
||||
return hfilesSize.get();
|
||||
}
|
||||
|
||||
/** @return the total size of the store files in the archive */
|
||||
public long getArchivedStoreFileSize() {
|
||||
return hfileArchiveSize.get();
|
||||
return hfilesArchiveSize.get();
|
||||
}
|
||||
|
||||
/** @return the total size of the store files in the mob store*/
|
||||
public long getMobStoreFilesSize() { return hfilesMobSize.get(); }
|
||||
|
||||
/** @return the percentage of the shared store files */
|
||||
public float getSharedStoreFilePercentage() {
|
||||
return ((float)hfileSize.get() / (hfileSize.get() + hfileArchiveSize.get())) * 100;
|
||||
return ((float) hfilesSize.get() / (getStoreFilesSize())) * 100;
|
||||
}
|
||||
|
||||
/** @return the percentage of the mob store files */
|
||||
public float getMobStoreFilePercentage() {
|
||||
return ((float) hfilesMobSize.get() / (getStoreFilesSize())) * 100;
|
||||
}
|
||||
|
||||
/** @return the total log size */
|
||||
|
@ -221,11 +234,15 @@ public final class SnapshotInfo extends Configured implements Tool {
|
|||
try {
|
||||
if ((inArchive = fs.exists(link.getArchivePath()))) {
|
||||
size = fs.getFileStatus(link.getArchivePath()).getLen();
|
||||
hfileArchiveSize.addAndGet(size);
|
||||
hfileArchiveCount.incrementAndGet();
|
||||
hfilesArchiveSize.addAndGet(size);
|
||||
hfilesArchiveCount.incrementAndGet();
|
||||
} else if (inArchive = fs.exists(link.getMobPath())) {
|
||||
size = fs.getFileStatus(link.getMobPath()).getLen();
|
||||
hfilesMobSize.addAndGet(size);
|
||||
hfilesMobCount.incrementAndGet();
|
||||
} else {
|
||||
size = link.getFileStatus(fs).getLen();
|
||||
hfileSize.addAndGet(size);
|
||||
hfilesSize.addAndGet(size);
|
||||
hfilesCount.incrementAndGet();
|
||||
}
|
||||
isCorrupted = (storeFile.hasFileSize() && storeFile.getFileSize() != size);
|
||||
|
@ -441,11 +458,15 @@ public final class SnapshotInfo extends Configured implements Tool {
|
|||
}
|
||||
|
||||
if (showStats) {
|
||||
System.out.printf("%d HFiles (%d in archive), total size %s (%.2f%% %s shared with the source table)%n",
|
||||
System.out.printf("%d HFiles (%d in archive, %d in mob storage), total size %s " +
|
||||
"(%.2f%% %s shared with the source table, %.2f%% %s in mob dir)%n",
|
||||
stats.getStoreFilesCount(), stats.getArchivedStoreFilesCount(),
|
||||
stats.getMobStoreFilesCount(),
|
||||
fileSizeToString(stats.getStoreFilesSize()),
|
||||
stats.getSharedStoreFilePercentage(),
|
||||
fileSizeToString(stats.getSharedStoreFilesSize())
|
||||
fileSizeToString(stats.getSharedStoreFilesSize()),
|
||||
stats.getMobStoreFilePercentage(),
|
||||
fileSizeToString(stats.getMobStoreFilesSize())
|
||||
);
|
||||
System.out.printf("%d Logs, total size %s%n",
|
||||
stats.getLogsCount(), fileSizeToString(stats.getLogsSize()));
|
||||
|
|
|
@ -18,8 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.snapshot;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
|
@ -34,24 +34,27 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
||||
/**
|
||||
|
@ -151,6 +154,60 @@ public class SnapshotManifest {
|
|||
}
|
||||
}
|
||||
|
||||
public void addMobRegion(HRegionInfo regionInfo, HColumnDescriptor[] hcds) throws IOException {
|
||||
// 0. Get the ManifestBuilder/RegionVisitor
|
||||
RegionVisitor visitor = createRegionVisitor(desc);
|
||||
|
||||
// 1. dump region meta info into the snapshot directory
|
||||
LOG.debug("Storing mob region '" + regionInfo + "' region-info for snapshot.");
|
||||
Object regionData = visitor.regionOpen(regionInfo);
|
||||
monitor.rethrowException();
|
||||
|
||||
// 2. iterate through all the stores in the region
|
||||
LOG.debug("Creating references for mob files");
|
||||
|
||||
Path mobRegionPath = MobUtils.getMobRegionPath(conf, regionInfo.getTable());
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
// 2.1. build the snapshot reference for the store if it's a mob store
|
||||
if (!hcd.isMobEnabled()) {
|
||||
continue;
|
||||
}
|
||||
Object familyData = visitor.familyOpen(regionData, hcd.getName());
|
||||
monitor.rethrowException();
|
||||
|
||||
Path storePath = MobUtils.getMobFamilyPath(mobRegionPath, hcd.getNameAsString());
|
||||
if (!fs.exists(storePath)) {
|
||||
continue;
|
||||
}
|
||||
FileStatus[] stats = fs.listStatus(storePath);
|
||||
if (stats == null) {
|
||||
continue;
|
||||
}
|
||||
List<StoreFileInfo> storeFiles = new ArrayList<StoreFileInfo>();
|
||||
for (FileStatus stat : stats) {
|
||||
storeFiles.add(new StoreFileInfo(conf, fs, stat));
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Adding snapshot references for " + storeFiles + " mob files");
|
||||
}
|
||||
|
||||
// 2.2. iterate through all the mob files and create "references".
|
||||
for (int i = 0, sz = storeFiles.size(); i < sz; i++) {
|
||||
StoreFileInfo storeFile = storeFiles.get(i);
|
||||
monitor.rethrowException();
|
||||
|
||||
// create "reference" to this store file.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Adding reference for mob file (" + (i + 1) + "/" + sz + "): "
|
||||
+ storeFile.getPath());
|
||||
}
|
||||
visitor.storeFile(regionData, familyData, storeFile);
|
||||
}
|
||||
visitor.familyClose(regionData, familyData);
|
||||
}
|
||||
visitor.regionClose(regionData);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a 'manifest' for the specified region, by reading directly from the HRegion object.
|
||||
* This is used by the "online snapshot" when the table is enabled.
|
||||
|
@ -199,55 +256,83 @@ public class SnapshotManifest {
|
|||
// 0. Get the ManifestBuilder/RegionVisitor
|
||||
RegionVisitor visitor = createRegionVisitor(desc);
|
||||
|
||||
// Open the RegionFS
|
||||
HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs,
|
||||
tableDir, regionInfo, true);
|
||||
monitor.rethrowException();
|
||||
boolean isMobRegion = MobUtils.isMobRegionInfo(regionInfo);
|
||||
try {
|
||||
// Open the RegionFS
|
||||
HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs,
|
||||
tableDir, regionInfo, true);
|
||||
monitor.rethrowException();
|
||||
|
||||
// 1. dump region meta info into the snapshot directory
|
||||
LOG.debug("Storing region-info for snapshot.");
|
||||
Object regionData = visitor.regionOpen(regionInfo);
|
||||
monitor.rethrowException();
|
||||
// 1. dump region meta info into the snapshot directory
|
||||
LOG.debug("Storing region-info for snapshot.");
|
||||
Object regionData = visitor.regionOpen(regionInfo);
|
||||
monitor.rethrowException();
|
||||
|
||||
// 2. iterate through all the stores in the region
|
||||
LOG.debug("Creating references for hfiles");
|
||||
// 2. iterate through all the stores in the region
|
||||
LOG.debug("Creating references for hfiles");
|
||||
|
||||
// This ensures that we have an atomic view of the directory as long as we have < ls limit
|
||||
// (batch size of the files in a directory) on the namenode. Otherwise, we get back the files in
|
||||
// batches and may miss files being added/deleted. This could be more robust (iteratively
|
||||
// checking to see if we have all the files until we are sure), but the limit is currently 1000
|
||||
// files/batch, far more than the number of store files under a single column family.
|
||||
Collection<String> familyNames = regionFs.getFamilies();
|
||||
if (familyNames != null) {
|
||||
for (String familyName: familyNames) {
|
||||
Object familyData = visitor.familyOpen(regionData, Bytes.toBytes(familyName));
|
||||
monitor.rethrowException();
|
||||
|
||||
Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(familyName);
|
||||
if (storeFiles == null) {
|
||||
LOG.debug("No files under family: " + familyName);
|
||||
continue;
|
||||
}
|
||||
|
||||
// 2.1. build the snapshot reference for the store
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Adding snapshot references for " + storeFiles + " hfiles");
|
||||
}
|
||||
|
||||
// 2.2. iterate through all the store's files and create "references".
|
||||
int i = 0;
|
||||
int sz = storeFiles.size();
|
||||
for (StoreFileInfo storeFile: storeFiles) {
|
||||
// This ensures that we have an atomic view of the directory as long as we have < ls limit
|
||||
// (batch size of the files in a directory) on the namenode. Otherwise, we get back the files
|
||||
// in batches and may miss files being added/deleted. This could be more robust (iteratively
|
||||
// checking to see if we have all the files until we are sure), but the limit is currently
|
||||
// 1000 files/batch, far more than the number of store files under a single column family.
|
||||
Collection<String> familyNames = regionFs.getFamilies();
|
||||
if (familyNames != null) {
|
||||
for (String familyName: familyNames) {
|
||||
Object familyData = visitor.familyOpen(regionData, Bytes.toBytes(familyName));
|
||||
monitor.rethrowException();
|
||||
|
||||
// create "reference" to this store file.
|
||||
LOG.debug("Adding reference for file ("+ (++i) +"/" + sz + "): " + storeFile.getPath());
|
||||
visitor.storeFile(regionData, familyData, storeFile);
|
||||
Collection<StoreFileInfo> storeFiles = null;
|
||||
if (isMobRegion) {
|
||||
Path regionPath = MobUtils.getMobRegionPath(conf, regionInfo.getTable());
|
||||
Path storePath = MobUtils.getMobFamilyPath(regionPath, familyName);
|
||||
if (!fs.exists(storePath)) {
|
||||
continue;
|
||||
}
|
||||
FileStatus[] stats = fs.listStatus(storePath);
|
||||
if (stats == null) {
|
||||
continue;
|
||||
}
|
||||
storeFiles = new ArrayList<StoreFileInfo>();
|
||||
for (FileStatus stat : stats) {
|
||||
storeFiles.add(new StoreFileInfo(conf, fs, stat));
|
||||
}
|
||||
} else {
|
||||
storeFiles = regionFs.getStoreFiles(familyName);
|
||||
}
|
||||
if (storeFiles == null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("No files under family: " + familyName);
|
||||
}
|
||||
continue;
|
||||
}
|
||||
|
||||
// 2.1. build the snapshot reference for the store
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Adding snapshot references for " + storeFiles + " hfiles");
|
||||
}
|
||||
|
||||
// 2.2. iterate through all the store's files and create "references".
|
||||
int i = 0;
|
||||
int sz = storeFiles.size();
|
||||
for (StoreFileInfo storeFile: storeFiles) {
|
||||
monitor.rethrowException();
|
||||
|
||||
// create "reference" to this store file.
|
||||
LOG.debug("Adding reference for file (" + (++i) + "/" + sz + "): "
|
||||
+ storeFile.getPath());
|
||||
visitor.storeFile(regionData, familyData, storeFile);
|
||||
}
|
||||
visitor.familyClose(regionData, familyData);
|
||||
}
|
||||
visitor.familyClose(regionData, familyData);
|
||||
}
|
||||
visitor.regionClose(regionData);
|
||||
} catch (IOException e) {
|
||||
// the mob directory might not be created yet, so do nothing when it is a mob region
|
||||
if (!isMobRegion) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
visitor.regionClose(regionData);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -39,11 +39,13 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.util.FSVisitor;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
|
||||
/**
|
||||
* Utility methods for interacting with the snapshot referenced files.
|
||||
|
@ -296,7 +298,15 @@ public final class SnapshotReferenceUtil {
|
|||
}
|
||||
|
||||
// check if the linked file exists (in the archive, or in the table dir)
|
||||
HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, linkPath);
|
||||
HFileLink link = null;
|
||||
if (MobUtils.isMobRegionInfo(regionInfo)) {
|
||||
// for mob region
|
||||
link = HFileLink.buildFromHFileLinkPattern(MobUtils.getQualifiedMobRootDir(conf),
|
||||
HFileArchiveUtil.getArchivePath(conf), linkPath);
|
||||
} else {
|
||||
// not mob region
|
||||
link = HFileLink.buildFromHFileLinkPattern(conf, linkPath);
|
||||
}
|
||||
try {
|
||||
FileStatus fstat = link.getFileStatus(fs);
|
||||
if (storeFile.hasFileSize() && storeFile.getFileSize() != fstat.getLen()) {
|
||||
|
|
|
@ -64,10 +64,8 @@ public class HFileArchiveUtil {
|
|||
HRegionInfo region,
|
||||
Path tabledir,
|
||||
byte[] family) throws IOException {
|
||||
TableName tableName =
|
||||
FSUtils.getTableName(tabledir);
|
||||
Path rootDir = FSUtils.getRootDir(conf);
|
||||
Path tableArchiveDir = getTableArchivePath(rootDir, tableName);
|
||||
Path tableArchiveDir = getTableArchivePath(rootDir, region.getTable());
|
||||
return HStore.getStoreHomedir(tableArchiveDir, region, family);
|
||||
}
|
||||
|
||||
|
|
|
@ -39,9 +39,11 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.FSUtils.FamilyDirFilter;
|
||||
import org.apache.hadoop.hbase.util.FSUtils.HFileFilter;
|
||||
|
@ -68,8 +70,13 @@ public class HFileCorruptionChecker {
|
|||
final Set<Path> failures = new ConcurrentSkipListSet<Path>();
|
||||
final Set<Path> quarantined = new ConcurrentSkipListSet<Path>();
|
||||
final Set<Path> missing = new ConcurrentSkipListSet<Path>();
|
||||
final Set<Path> corruptedMobFiles = new ConcurrentSkipListSet<Path>();
|
||||
final Set<Path> failureMobFiles = new ConcurrentSkipListSet<Path>();
|
||||
final Set<Path> missedMobFiles = new ConcurrentSkipListSet<Path>();
|
||||
final Set<Path> quarantinedMobFiles = new ConcurrentSkipListSet<Path>();
|
||||
final boolean inQuarantineMode;
|
||||
final AtomicInteger hfilesChecked = new AtomicInteger();
|
||||
final AtomicInteger mobFilesChecked = new AtomicInteger();
|
||||
|
||||
public HFileCorruptionChecker(Configuration conf, ExecutorService executor,
|
||||
boolean quarantine) throws IOException {
|
||||
|
@ -176,6 +183,109 @@ public class HFileCorruptionChecker {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check all files in a mob column family dir.
|
||||
*
|
||||
* @param cfDir
|
||||
* mob column family directory
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void checkMobColFamDir(Path cfDir) throws IOException {
|
||||
FileStatus[] hfs = null;
|
||||
try {
|
||||
hfs = fs.listStatus(cfDir, new HFileFilter(fs)); // use same filter as scanner.
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
// Hadoop 0.23+ listStatus semantics throws an exception if the path does not exist.
|
||||
LOG.warn("Mob colfam Directory " + cfDir +
|
||||
" does not exist. Likely the table is deleted. Skipping.");
|
||||
missedMobFiles.add(cfDir);
|
||||
return;
|
||||
}
|
||||
|
||||
// Hadoop 1.0 listStatus does not throw an exception if the path does not exist.
|
||||
if (hfs.length == 0 && !fs.exists(cfDir)) {
|
||||
LOG.warn("Mob colfam Directory " + cfDir +
|
||||
" does not exist. Likely the table is deleted. Skipping.");
|
||||
missedMobFiles.add(cfDir);
|
||||
return;
|
||||
}
|
||||
for (FileStatus hfFs : hfs) {
|
||||
Path hf = hfFs.getPath();
|
||||
checkMobFile(hf);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks a path to see if it is a valid mob file.
|
||||
*
|
||||
* @param p
|
||||
* full Path to a mob file.
|
||||
* @throws IOException
|
||||
* This is a connectivity related exception
|
||||
*/
|
||||
protected void checkMobFile(Path p) throws IOException {
|
||||
HFile.Reader r = null;
|
||||
try {
|
||||
r = HFile.createReader(fs, p, cacheConf, conf);
|
||||
} catch (CorruptHFileException che) {
|
||||
LOG.warn("Found corrupt mob file " + p, che);
|
||||
corruptedMobFiles.add(p);
|
||||
if (inQuarantineMode) {
|
||||
Path dest = createQuarantinePath(p);
|
||||
LOG.warn("Quarantining corrupt mob file " + p + " into " + dest);
|
||||
boolean success = fs.mkdirs(dest.getParent());
|
||||
success = success ? fs.rename(p, dest): false;
|
||||
if (!success) {
|
||||
failureMobFiles.add(p);
|
||||
} else {
|
||||
quarantinedMobFiles.add(dest);
|
||||
}
|
||||
}
|
||||
return;
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
LOG.warn("Mob file " + p + " was missing. Likely removed due to compaction?");
|
||||
missedMobFiles.add(p);
|
||||
} finally {
|
||||
mobFilesChecked.addAndGet(1);
|
||||
if (r != null) {
|
||||
r.close(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks all the mob files of a table.
|
||||
* @param regionDir The mob region directory
|
||||
* @throws IOException
|
||||
*/
|
||||
private void checkMobRegionDir(Path regionDir) throws IOException {
|
||||
if (!fs.exists(regionDir)) {
|
||||
return;
|
||||
}
|
||||
FileStatus[] hfs = null;
|
||||
try {
|
||||
hfs = fs.listStatus(regionDir, new FamilyDirFilter(fs));
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
// Hadoop 0.23+ listStatus semantics throws an exception if the path does not exist.
|
||||
LOG.warn("Mob directory " + regionDir
|
||||
+ " does not exist. Likely the table is deleted. Skipping.");
|
||||
missedMobFiles.add(regionDir);
|
||||
return;
|
||||
}
|
||||
|
||||
// Hadoop 1.0 listStatus does not throw an exception if the path does not exist.
|
||||
if (hfs.length == 0 && !fs.exists(regionDir)) {
|
||||
LOG.warn("Mob directory " + regionDir
|
||||
+ " does not exist. Likely the table is deleted. Skipping.");
|
||||
missedMobFiles.add(regionDir);
|
||||
return;
|
||||
}
|
||||
for (FileStatus hfFs : hfs) {
|
||||
Path hf = hfFs.getPath();
|
||||
checkMobColFamDir(hf);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check all column families in a region dir.
|
||||
*
|
||||
|
@ -236,6 +346,8 @@ public class HFileCorruptionChecker {
|
|||
rdcs.add(work);
|
||||
}
|
||||
|
||||
// add mob region
|
||||
rdcs.add(createMobRegionDirChecker(tableDir));
|
||||
// Submit and wait for completion
|
||||
try {
|
||||
rdFutures = executor.invokeAll(rdcs);
|
||||
|
@ -292,6 +404,34 @@ public class HFileCorruptionChecker {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* An individual work item for parallelized mob dir processing. This is
|
||||
* intentionally an inner class so it can use the shared error sets and fs.
|
||||
*/
|
||||
private class MobRegionDirChecker extends RegionDirChecker {
|
||||
|
||||
MobRegionDirChecker(Path regionDir) {
|
||||
super(regionDir);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void call() throws IOException {
|
||||
checkMobRegionDir(regionDir);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an instance of MobRegionDirChecker.
|
||||
* @param tableDir The current table directory.
|
||||
* @return An instance of MobRegionDirChecker.
|
||||
*/
|
||||
private MobRegionDirChecker createMobRegionDirChecker(Path tableDir) {
|
||||
TableName tableName = FSUtils.getTableName(tableDir);
|
||||
Path mobDir = MobUtils.getMobRegionPath(conf, tableName);
|
||||
return new MobRegionDirChecker(mobDir);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check the specified table dirs for bad hfiles.
|
||||
*/
|
||||
|
@ -337,6 +477,42 @@ public class HFileCorruptionChecker {
|
|||
return new HashSet<Path>(missing);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the set of check failure mob file paths after checkTables is called.
|
||||
*/
|
||||
public Collection<Path> getFailureMobFiles() {
|
||||
return new HashSet<Path>(failureMobFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the set of corrupted mob file paths after checkTables is called.
|
||||
*/
|
||||
public Collection<Path> getCorruptedMobFiles() {
|
||||
return new HashSet<Path>(corruptedMobFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return number of mob files checked in the last HfileCorruptionChecker run
|
||||
*/
|
||||
public int getMobFilesChecked() {
|
||||
return mobFilesChecked.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the set of successfully quarantined paths after checkTables is called.
|
||||
*/
|
||||
public Collection<Path> getQuarantinedMobFiles() {
|
||||
return new HashSet<Path>(quarantinedMobFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the set of paths that were missing. Likely due to table deletion or
|
||||
* deletion/moves from compaction.
|
||||
*/
|
||||
public Collection<Path> getMissedMobFiles() {
|
||||
return new HashSet<Path>(missedMobFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* Print a human readable summary of hfile quarantining operations.
|
||||
* @param out
|
||||
|
@ -363,10 +539,31 @@ public class HFileCorruptionChecker {
|
|||
String fixedState = (corrupted.size() == quarantined.size()) ? "OK"
|
||||
: "CORRUPTED";
|
||||
|
||||
// print mob-related report
|
||||
if (inQuarantineMode) {
|
||||
out.print(" Mob files successfully quarantined: " + quarantinedMobFiles.size());
|
||||
for (Path sq : quarantinedMobFiles) {
|
||||
out.print(" " + sq);
|
||||
}
|
||||
out.print(" Mob files failed quarantine: " + failureMobFiles.size());
|
||||
for (Path fq : failureMobFiles) {
|
||||
out.print(" " + fq);
|
||||
}
|
||||
}
|
||||
out.print(" Mob files moved while checking: " + missedMobFiles.size());
|
||||
for (Path mq : missedMobFiles) {
|
||||
out.print(" " + mq);
|
||||
}
|
||||
String initialMobState = (corruptedMobFiles.size() == 0) ? "OK" : "CORRUPTED";
|
||||
String fixedMobState = (corruptedMobFiles.size() == quarantinedMobFiles.size()) ? "OK"
|
||||
: "CORRUPTED";
|
||||
|
||||
if (inQuarantineMode) {
|
||||
out.print("Summary: " + initialState + " => " + fixedState);
|
||||
out.print("Mob summary: " + initialMobState + " => " + fixedMobState);
|
||||
} else {
|
||||
out.print("Summary: " + initialState);
|
||||
out.print("Mob summary: " + initialMobState);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -75,12 +75,21 @@ public class TestAcidGuarantees implements Tool {
|
|||
// when run as main
|
||||
private Configuration conf;
|
||||
|
||||
private void createTableIfMissing()
|
||||
private void createTableIfMissing(boolean useMob)
|
||||
throws IOException {
|
||||
try {
|
||||
util.createTable(TABLE_NAME, FAMILIES);
|
||||
} catch (TableExistsException tee) {
|
||||
}
|
||||
|
||||
if (useMob) {
|
||||
HTableDescriptor htd = util.getHBaseAdmin().getTableDescriptor(TABLE_NAME);
|
||||
HColumnDescriptor hcd = htd.getColumnFamilies()[0];
|
||||
// force mob enabled such that all data is mob data
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(4);
|
||||
util.getHBaseAdmin().modifyColumn(TABLE_NAME, hcd);
|
||||
}
|
||||
}
|
||||
|
||||
public TestAcidGuarantees() {
|
||||
|
@ -90,6 +99,7 @@ public class TestAcidGuarantees implements Tool {
|
|||
// prevent aggressive region split
|
||||
conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
ConstantSizeRegionSplitPolicy.class.getName());
|
||||
conf.setInt("hfile.format.version", 3); // for mob tests
|
||||
util = new HBaseTestingUtility(conf);
|
||||
}
|
||||
|
||||
|
@ -265,7 +275,19 @@ public class TestAcidGuarantees implements Tool {
|
|||
int numScanners,
|
||||
int numUniqueRows,
|
||||
final boolean systemTest) throws Exception {
|
||||
createTableIfMissing();
|
||||
runTestAtomicity(millisToRun, numWriters, numGetters, numScanners, numUniqueRows, systemTest,
|
||||
false);
|
||||
}
|
||||
|
||||
public void runTestAtomicity(long millisToRun,
|
||||
int numWriters,
|
||||
int numGetters,
|
||||
int numScanners,
|
||||
int numUniqueRows,
|
||||
final boolean systemTest,
|
||||
final boolean useMob) throws Exception {
|
||||
|
||||
createTableIfMissing(useMob);
|
||||
TestContext ctx = new TestContext(util.getConfiguration());
|
||||
|
||||
byte rows[][] = new byte[numUniqueRows][];
|
||||
|
@ -365,6 +387,42 @@ public class TestAcidGuarantees implements Tool {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMobGetAtomicity() throws Exception {
|
||||
util.startMiniCluster(1);
|
||||
try {
|
||||
boolean systemTest = false;
|
||||
boolean useMob = true;
|
||||
runTestAtomicity(20000, 5, 5, 0, 3, systemTest, useMob);
|
||||
} finally {
|
||||
util.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMobScanAtomicity() throws Exception {
|
||||
util.startMiniCluster(1);
|
||||
try {
|
||||
boolean systemTest = false;
|
||||
boolean useMob = true;
|
||||
runTestAtomicity(20000, 5, 0, 5, 3, systemTest, useMob);
|
||||
} finally {
|
||||
util.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMobMixedAtomicity() throws Exception {
|
||||
util.startMiniCluster(1);
|
||||
try {
|
||||
boolean systemTest = false;
|
||||
boolean useMob = true;
|
||||
runTestAtomicity(20000, 5, 2, 2, 3, systemTest, useMob);
|
||||
} finally {
|
||||
util.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
// Tool interface
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
|
@ -387,7 +445,9 @@ public class TestAcidGuarantees implements Tool {
|
|||
int numGetters = c.getInt("numGetters", 2);
|
||||
int numScanners = c.getInt("numScanners", 2);
|
||||
int numUniqueRows = c.getInt("numUniqueRows", 3);
|
||||
runTestAtomicity(millis, numWriters, numGetters, numScanners, numUniqueRows, true);
|
||||
boolean useMob = c.getBoolean("useMob",false);
|
||||
assert useMob && c.getInt("hfile.format.version", 2) == 3 : "Mob runs must use hfile v3";
|
||||
runTestAtomicity(millis, numWriters, numGetters, numScanners, numUniqueRows, true, useMob);
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,252 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Test clone snapshots from the client
|
||||
*/
|
||||
@Category({LargeTests.class, ClientTests.class})
|
||||
public class TestMobCloneSnapshotFromClient {
|
||||
final Log LOG = LogFactory.getLog(getClass());
|
||||
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
private final byte[] FAMILY = Bytes.toBytes("cf");
|
||||
|
||||
private byte[] emptySnapshot;
|
||||
private byte[] snapshotName0;
|
||||
private byte[] snapshotName1;
|
||||
private byte[] snapshotName2;
|
||||
private int snapshot0Rows;
|
||||
private int snapshot1Rows;
|
||||
private TableName tableName;
|
||||
private Admin admin;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
|
||||
TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
|
||||
TEST_UTIL.getConfiguration().setBoolean(
|
||||
"hbase.master.enabletable.roundrobin", true);
|
||||
TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
|
||||
TEST_UTIL.startMiniCluster(3);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the tests with a table filled with some data
|
||||
* and two snapshots (snapshotName0, snapshotName1) of different states.
|
||||
* The tableName, snapshotNames and the number of rows in the snapshot are initialized.
|
||||
*/
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
this.admin = TEST_UTIL.getHBaseAdmin();
|
||||
|
||||
long tid = System.currentTimeMillis();
|
||||
tableName = TableName.valueOf("testtb-" + tid);
|
||||
emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
|
||||
snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
|
||||
snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
|
||||
snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
|
||||
|
||||
// create Table and disable it
|
||||
MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
|
||||
admin.disableTable(tableName);
|
||||
|
||||
// take an empty snapshot
|
||||
admin.snapshot(emptySnapshot, tableName);
|
||||
|
||||
Connection c = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
|
||||
Table table = c.getTable(tableName);
|
||||
try {
|
||||
// enable table and insert data
|
||||
admin.enableTable(tableName);
|
||||
SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
|
||||
snapshot0Rows = MobSnapshotTestingUtils.countMobRows(table);
|
||||
admin.disableTable(tableName);
|
||||
|
||||
// take a snapshot
|
||||
admin.snapshot(snapshotName0, tableName);
|
||||
|
||||
// enable table and insert more data
|
||||
admin.enableTable(tableName);
|
||||
SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
|
||||
snapshot1Rows = MobSnapshotTestingUtils.countMobRows(table);
|
||||
admin.disableTable(tableName);
|
||||
|
||||
// take a snapshot of the updated table
|
||||
admin.snapshot(snapshotName1, tableName);
|
||||
|
||||
// re-enable table
|
||||
admin.enableTable(tableName);
|
||||
} finally {
|
||||
table.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected int getNumReplicas() {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
if (admin.tableExists(tableName)) {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
}
|
||||
SnapshotTestingUtils.deleteAllSnapshots(admin);
|
||||
SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
|
||||
}
|
||||
|
||||
@Test(expected=SnapshotDoesNotExistException.class)
|
||||
public void testCloneNonExistentSnapshot() throws IOException, InterruptedException {
|
||||
String snapshotName = "random-snapshot-" + System.currentTimeMillis();
|
||||
TableName tableName = TableName.valueOf("random-table-" + System.currentTimeMillis());
|
||||
admin.cloneSnapshot(snapshotName, tableName);
|
||||
}
|
||||
|
||||
@Test(expected = NamespaceNotFoundException.class)
|
||||
public void testCloneOnMissingNamespace() throws IOException, InterruptedException {
|
||||
TableName clonedTableName = TableName.valueOf("unknownNS:clonetb");
|
||||
admin.cloneSnapshot(snapshotName1, clonedTableName);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloneSnapshot() throws IOException, InterruptedException {
|
||||
TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis());
|
||||
testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
|
||||
testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
|
||||
testCloneSnapshot(clonedTableName, emptySnapshot, 0);
|
||||
}
|
||||
|
||||
private void testCloneSnapshot(final TableName tableName, final byte[] snapshotName,
|
||||
int snapshotRows) throws IOException, InterruptedException {
|
||||
// create a new table from snapshot
|
||||
admin.cloneSnapshot(snapshotName, tableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshotRows);
|
||||
|
||||
verifyReplicasCameOnline(tableName);
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
}
|
||||
|
||||
protected void verifyReplicasCameOnline(TableName tableName) throws IOException {
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloneSnapshotCrossNamespace() throws IOException, InterruptedException {
|
||||
String nsName = "testCloneSnapshotCrossNamespace";
|
||||
admin.createNamespace(NamespaceDescriptor.create(nsName).build());
|
||||
TableName clonedTableName =
|
||||
TableName.valueOf(nsName, "clonedtb-" + System.currentTimeMillis());
|
||||
testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
|
||||
testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
|
||||
testCloneSnapshot(clonedTableName, emptySnapshot, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that tables created from the snapshot are still alive after source table deletion.
|
||||
*/
|
||||
@Test
|
||||
public void testCloneLinksAfterDelete() throws IOException, InterruptedException {
|
||||
// Clone a table from the first snapshot
|
||||
TableName clonedTableName = TableName.valueOf("clonedtb1-" + System.currentTimeMillis());
|
||||
admin.cloneSnapshot(snapshotName0, clonedTableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
|
||||
|
||||
// Take a snapshot of this cloned table.
|
||||
admin.disableTable(clonedTableName);
|
||||
admin.snapshot(snapshotName2, clonedTableName);
|
||||
|
||||
// Clone the snapshot of the cloned table
|
||||
TableName clonedTableName2 = TableName.valueOf("clonedtb2-" + System.currentTimeMillis());
|
||||
admin.cloneSnapshot(snapshotName2, clonedTableName2);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
|
||||
admin.disableTable(clonedTableName2);
|
||||
|
||||
// Remove the original table
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
waitCleanerRun();
|
||||
|
||||
// Verify the first cloned table
|
||||
admin.enableTable(clonedTableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
|
||||
|
||||
// Verify the second cloned table
|
||||
admin.enableTable(clonedTableName2);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
|
||||
admin.disableTable(clonedTableName2);
|
||||
|
||||
// Delete the first cloned table
|
||||
TEST_UTIL.deleteTable(clonedTableName);
|
||||
waitCleanerRun();
|
||||
|
||||
// Verify the second cloned table
|
||||
admin.enableTable(clonedTableName2);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
|
||||
|
||||
// Clone a new table from cloned
|
||||
TableName clonedTableName3 = TableName.valueOf("clonedtb3-" + System.currentTimeMillis());
|
||||
admin.cloneSnapshot(snapshotName2, clonedTableName3);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName3, snapshot0Rows);
|
||||
|
||||
// Delete the cloned tables
|
||||
TEST_UTIL.deleteTable(clonedTableName2);
|
||||
TEST_UTIL.deleteTable(clonedTableName3);
|
||||
admin.deleteSnapshot(snapshotName2);
|
||||
}
|
||||
|
||||
// ==========================================================================
|
||||
// Helpers
|
||||
// ==========================================================================
|
||||
|
||||
private void waitCleanerRun() throws InterruptedException {
|
||||
TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,306 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
|
||||
import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
|
||||
import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Test restore snapshots from the client
|
||||
*/
|
||||
@Category({ClientTests.class, LargeTests.class})
|
||||
public class TestMobRestoreSnapshotFromClient {
|
||||
final Log LOG = LogFactory.getLog(getClass());
|
||||
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
private final byte[] FAMILY = Bytes.toBytes("cf");
|
||||
|
||||
private byte[] emptySnapshot;
|
||||
private byte[] snapshotName0;
|
||||
private byte[] snapshotName1;
|
||||
private byte[] snapshotName2;
|
||||
private int snapshot0Rows;
|
||||
private int snapshot1Rows;
|
||||
private TableName tableName;
|
||||
private Admin admin;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
|
||||
TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
|
||||
TEST_UTIL.getConfiguration().setBoolean(
|
||||
"hbase.master.enabletable.roundrobin", true);
|
||||
TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
|
||||
TEST_UTIL.startMiniCluster(3);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the tests with a table filled with some data
|
||||
* and two snapshots (snapshotName0, snapshotName1) of different states.
|
||||
* The tableName, snapshotNames and the number of rows in the snapshot are initialized.
|
||||
*/
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
this.admin = TEST_UTIL.getHBaseAdmin();
|
||||
|
||||
long tid = System.currentTimeMillis();
|
||||
tableName =
|
||||
TableName.valueOf("testtb-" + tid);
|
||||
emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
|
||||
snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
|
||||
snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
|
||||
snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
|
||||
|
||||
// create Table and disable it
|
||||
MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
|
||||
|
||||
admin.disableTable(tableName);
|
||||
|
||||
// take an empty snapshot
|
||||
admin.snapshot(emptySnapshot, tableName);
|
||||
|
||||
Table table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
|
||||
.getTable(tableName);
|
||||
// enable table and insert data
|
||||
admin.enableTable(tableName);
|
||||
SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
|
||||
snapshot0Rows = MobSnapshotTestingUtils.countMobRows(table);
|
||||
admin.disableTable(tableName);
|
||||
|
||||
// take a snapshot
|
||||
admin.snapshot(snapshotName0, tableName);
|
||||
|
||||
// enable table and insert more data
|
||||
admin.enableTable(tableName);
|
||||
SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
|
||||
snapshot1Rows = MobSnapshotTestingUtils.countMobRows(table);
|
||||
table.close();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
|
||||
SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRestoreSnapshot() throws IOException {
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot1Rows);
|
||||
admin.disableTable(tableName);
|
||||
admin.snapshot(snapshotName1, tableName);
|
||||
// Restore from snapshot-0
|
||||
admin.restoreSnapshot(snapshotName0);
|
||||
admin.enableTable(tableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot0Rows);
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
|
||||
|
||||
// Restore from emptySnapshot
|
||||
admin.disableTable(tableName);
|
||||
admin.restoreSnapshot(emptySnapshot);
|
||||
admin.enableTable(tableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, 0);
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
|
||||
|
||||
// Restore from snapshot-1
|
||||
admin.disableTable(tableName);
|
||||
admin.restoreSnapshot(snapshotName1);
|
||||
admin.enableTable(tableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot1Rows);
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
|
||||
|
||||
// Restore from snapshot-1
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
admin.restoreSnapshot(snapshotName1);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot1Rows);
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
|
||||
}
|
||||
|
||||
protected int getNumReplicas() {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRestoreSchemaChange() throws Exception {
|
||||
byte[] TEST_FAMILY2 = Bytes.toBytes("cf2");
|
||||
|
||||
Table table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
|
||||
.getTable(tableName);
|
||||
|
||||
// Add one column family and put some data in it
|
||||
admin.disableTable(tableName);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY2);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(3L);
|
||||
admin.addColumn(tableName, hcd);
|
||||
admin.enableTable(tableName);
|
||||
assertEquals(2, table.getTableDescriptor().getFamilies().size());
|
||||
HTableDescriptor htd = admin.getTableDescriptor(tableName);
|
||||
assertEquals(2, htd.getFamilies().size());
|
||||
SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, TEST_FAMILY2);
|
||||
long snapshot2Rows = snapshot1Rows + 500;
|
||||
assertEquals(snapshot2Rows, MobSnapshotTestingUtils.countMobRows(table));
|
||||
assertEquals(500, MobSnapshotTestingUtils.countMobRows(table, TEST_FAMILY2));
|
||||
Set<String> fsFamilies = getFamiliesFromFS(tableName);
|
||||
assertEquals(2, fsFamilies.size());
|
||||
|
||||
// Take a snapshot
|
||||
admin.disableTable(tableName);
|
||||
admin.snapshot(snapshotName2, tableName);
|
||||
|
||||
// Restore the snapshot (without the cf)
|
||||
admin.restoreSnapshot(snapshotName0);
|
||||
admin.enableTable(tableName);
|
||||
assertEquals(1, table.getTableDescriptor().getFamilies().size());
|
||||
try {
|
||||
MobSnapshotTestingUtils.countMobRows(table, TEST_FAMILY2);
|
||||
fail("family '" + Bytes.toString(TEST_FAMILY2) + "' should not exists");
|
||||
} catch (NoSuchColumnFamilyException e) {
|
||||
// expected
|
||||
}
|
||||
assertEquals(snapshot0Rows, MobSnapshotTestingUtils.countMobRows(table));
|
||||
htd = admin.getTableDescriptor(tableName);
|
||||
assertEquals(1, htd.getFamilies().size());
|
||||
fsFamilies = getFamiliesFromFS(tableName);
|
||||
assertEquals(1, fsFamilies.size());
|
||||
|
||||
// Restore back the snapshot (with the cf)
|
||||
admin.disableTable(tableName);
|
||||
admin.restoreSnapshot(snapshotName2);
|
||||
admin.enableTable(tableName);
|
||||
htd = admin.getTableDescriptor(tableName);
|
||||
assertEquals(2, htd.getFamilies().size());
|
||||
assertEquals(2, table.getTableDescriptor().getFamilies().size());
|
||||
assertEquals(500, MobSnapshotTestingUtils.countMobRows(table, TEST_FAMILY2));
|
||||
assertEquals(snapshot2Rows, MobSnapshotTestingUtils.countMobRows(table));
|
||||
fsFamilies = getFamiliesFromFS(tableName);
|
||||
assertEquals(2, fsFamilies.size());
|
||||
table.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloneSnapshotOfCloned() throws IOException, InterruptedException {
|
||||
TableName clonedTableName =
|
||||
TableName.valueOf("clonedtb-" + System.currentTimeMillis());
|
||||
admin.cloneSnapshot(snapshotName0, clonedTableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(clonedTableName, admin, getNumReplicas());
|
||||
admin.disableTable(clonedTableName);
|
||||
admin.snapshot(snapshotName2, clonedTableName);
|
||||
TEST_UTIL.deleteTable(clonedTableName);
|
||||
waitCleanerRun();
|
||||
|
||||
admin.cloneSnapshot(snapshotName2, clonedTableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(clonedTableName, admin, getNumReplicas());
|
||||
TEST_UTIL.deleteTable(clonedTableName);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloneAndRestoreSnapshot() throws IOException, InterruptedException {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
waitCleanerRun();
|
||||
|
||||
admin.cloneSnapshot(snapshotName0, tableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot0Rows);
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
|
||||
waitCleanerRun();
|
||||
|
||||
admin.disableTable(tableName);
|
||||
admin.restoreSnapshot(snapshotName0);
|
||||
admin.enableTable(tableName);
|
||||
MobSnapshotTestingUtils.verifyMobRowCount(TEST_UTIL, tableName, snapshot0Rows);
|
||||
SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCorruptedSnapshot() throws IOException, InterruptedException {
|
||||
SnapshotTestingUtils.corruptSnapshot(TEST_UTIL, Bytes.toString(snapshotName0));
|
||||
TableName cloneName = TableName.valueOf("corruptedClone-" + System.currentTimeMillis());
|
||||
try {
|
||||
admin.cloneSnapshot(snapshotName0, cloneName);
|
||||
fail("Expected CorruptedSnapshotException, got succeeded cloneSnapshot()");
|
||||
} catch (CorruptedSnapshotException e) {
|
||||
// Got the expected corruption exception.
|
||||
// check for no references of the cloned table.
|
||||
assertFalse(admin.tableExists(cloneName));
|
||||
} catch (Exception e) {
|
||||
fail("Expected CorruptedSnapshotException got: " + e);
|
||||
}
|
||||
}
|
||||
|
||||
// ==========================================================================
|
||||
// Helpers
|
||||
// ==========================================================================
|
||||
private void waitCleanerRun() throws InterruptedException {
|
||||
TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
|
||||
}
|
||||
|
||||
private Set<String> getFamiliesFromFS(final TableName tableName) throws IOException {
|
||||
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
|
||||
Set<String> families = new HashSet<String>();
|
||||
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
|
||||
for (Path regionDir: FSUtils.getRegionDirs(mfs.getFileSystem(), tableDir)) {
|
||||
for (Path familyDir: FSUtils.getFamilyDirs(mfs.getFileSystem(), regionDir)) {
|
||||
families.add(familyDir.getName());
|
||||
}
|
||||
}
|
||||
return families;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,435 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
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.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
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.master.cleaner.HFileCleaner;
|
||||
import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
|
||||
import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Test to verify that the cloned table is independent of the table from which it was cloned
|
||||
*/
|
||||
@Category(LargeTests.class)
|
||||
public class TestMobSnapshotCloneIndependence {
|
||||
private static final Log LOG = LogFactory.getLog(TestSnapshotCloneIndependence.class);
|
||||
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
private static final int NUM_RS = 2;
|
||||
private static final String STRING_TABLE_NAME = "test";
|
||||
private static final String TEST_FAM_STR = "fam";
|
||||
private static final byte[] TEST_FAM = Bytes.toBytes(TEST_FAM_STR);
|
||||
private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME);
|
||||
|
||||
/**
|
||||
* Setup the config for the cluster and start it
|
||||
* @throws Exception on failure
|
||||
*/
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
setupConf(UTIL.getConfiguration());
|
||||
UTIL.startMiniCluster(NUM_RS);
|
||||
}
|
||||
|
||||
private static void setupConf(Configuration conf) {
|
||||
// enable snapshot support
|
||||
conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
|
||||
// disable the ui
|
||||
conf.setInt("hbase.regionsever.info.port", -1);
|
||||
// change the flush size to a small amount, regulating number of store files
|
||||
conf.setInt("hbase.hregion.memstore.flush.size", 25000);
|
||||
// so make sure we get a compaction when doing a load, but keep around
|
||||
// some files in the store
|
||||
conf.setInt("hbase.hstore.compaction.min", 10);
|
||||
conf.setInt("hbase.hstore.compactionThreshold", 10);
|
||||
// block writes if we get to 12 store files
|
||||
conf.setInt("hbase.hstore.blockingStoreFiles", 12);
|
||||
conf.setInt("hbase.regionserver.msginterval", 100);
|
||||
conf.setBoolean("hbase.master.enabletable.roundrobin", true);
|
||||
// Avoid potentially aggressive splitting which would cause snapshot to fail
|
||||
conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
ConstantSizeRegionSplitPolicy.class.getName());
|
||||
conf.setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
|
||||
conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, SnapshotHFileCleaner.class.getName() + ","
|
||||
+ HFileLinkCleaner.class.getName());
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
MobSnapshotTestingUtils.createMobTable(UTIL, TableName.valueOf(STRING_TABLE_NAME), TEST_FAM);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
UTIL.deleteTable(TABLE_NAME);
|
||||
SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
|
||||
SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupTest() throws Exception {
|
||||
try {
|
||||
UTIL.shutdownMiniCluster();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("failure shutting down cluster", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that adding data to the cloned table will not affect the original, and vice-versa when
|
||||
* it is taken as an online snapshot.
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testOnlineSnapshotAppendIndependent() throws Exception {
|
||||
runTestSnapshotAppendIndependent(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that adding data to the cloned table will not affect the original, and vice-versa when
|
||||
* it is taken as an offline snapshot.
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testOfflineSnapshotAppendIndependent() throws Exception {
|
||||
runTestSnapshotAppendIndependent(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that adding metadata to the cloned table will not affect the original, and vice-versa
|
||||
* when it is taken as an online snapshot.
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testOnlineSnapshotMetadataChangesIndependent() throws Exception {
|
||||
runTestSnapshotMetadataChangesIndependent(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that adding netadata to the cloned table will not affect the original, and vice-versa
|
||||
* when is taken as an online snapshot.
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testOfflineSnapshotMetadataChangesIndependent() throws Exception {
|
||||
runTestSnapshotMetadataChangesIndependent(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that region operations, in this case splitting a region, are independent between the
|
||||
* cloned table and the original.
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testOfflineSnapshotRegionOperationsIndependent() throws Exception {
|
||||
runTestRegionOperationsIndependent(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that region operations, in this case splitting a region, are independent between the
|
||||
* cloned table and the original.
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testOnlineSnapshotRegionOperationsIndependent() throws Exception {
|
||||
runTestRegionOperationsIndependent(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify the mob cells still exist after the table to be cloned is deleted.
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testDeleteTableToBeCloned() throws Exception {
|
||||
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
|
||||
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
|
||||
TableName tn = TableName.valueOf("testDeleteTableToBeCloned");
|
||||
byte[] qf = Bytes.toBytes("qf");
|
||||
MobSnapshotTestingUtils.createMobTable(UTIL, tn, TEST_FAM);
|
||||
String row = "row";
|
||||
String value = "value";
|
||||
Put put = new Put(Bytes.toBytes(row));
|
||||
put.addColumn(TEST_FAM, qf, Bytes.toBytes(value));
|
||||
Admin admin = UTIL.getHBaseAdmin();
|
||||
BufferedMutator mutator = UTIL.getConnection().getBufferedMutator(tn);
|
||||
mutator.mutate(put);
|
||||
mutator.flush();
|
||||
admin.flush(tn);
|
||||
// Take a snapshot
|
||||
final String snapshotNameAsString = "snapshot_" + tn;
|
||||
byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
|
||||
Table table = ConnectionFactory.createConnection(UTIL.getConfiguration()).getTable(tn);
|
||||
Table clonedTable = null;
|
||||
try {
|
||||
SnapshotTestingUtils.createSnapshotAndValidate(admin, tn, TEST_FAM_STR, snapshotNameAsString,
|
||||
rootDir, fs, true);
|
||||
TableName cloneTableName = TableName.valueOf("test-clone-" + tn);
|
||||
admin.cloneSnapshot(snapshotName, cloneTableName);
|
||||
clonedTable = ConnectionFactory.createConnection(UTIL.getConfiguration()).getTable(
|
||||
cloneTableName);
|
||||
admin.deleteSnapshot(snapshotName);
|
||||
admin.disableTable(tn);
|
||||
admin.deleteTable(tn);
|
||||
// run the cleaner
|
||||
UTIL.getHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
|
||||
// make sure the mob cell exists
|
||||
Scan scan = new Scan();
|
||||
ResultScanner scanner = clonedTable.getScanner(scan);
|
||||
Result rs = scanner.next();
|
||||
Cell cell = rs.getColumnLatestCell(TEST_FAM, qf);
|
||||
Assert.assertEquals(value, Bytes.toString(CellUtil.cloneValue(cell)));
|
||||
Assert.assertNull(scanner.next());
|
||||
} finally {
|
||||
table.close();
|
||||
if (clonedTable != null) {
|
||||
clonedTable.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void waitOnSplit(Connection c, final Table t, int originalCount) throws Exception {
|
||||
for (int i = 0; i < 200; i++) {
|
||||
try {
|
||||
Thread.sleep(50);
|
||||
} catch (InterruptedException e) {
|
||||
// Restore the interrupted status
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
try (RegionLocator locator = c.getRegionLocator(t.getName())) {
|
||||
if (locator.getAllRegionLocations().size() > originalCount) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
throw new Exception("Split did not increase the number of regions");
|
||||
}
|
||||
|
||||
/*
|
||||
* Take a snapshot of a table, add data, and verify that this only
|
||||
* affects one table
|
||||
* @param online - Whether the table is online or not during the snapshot
|
||||
*/
|
||||
private void runTestSnapshotAppendIndependent(boolean online) throws Exception {
|
||||
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
|
||||
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
|
||||
|
||||
Admin admin = UTIL.getHBaseAdmin();
|
||||
final long startTime = System.currentTimeMillis();
|
||||
final TableName localTableName =
|
||||
TableName.valueOf(STRING_TABLE_NAME + startTime);
|
||||
|
||||
Table original = MobSnapshotTestingUtils.createMobTable(UTIL, localTableName, TEST_FAM);
|
||||
try {
|
||||
|
||||
SnapshotTestingUtils.loadData(UTIL, localTableName, 500, TEST_FAM);
|
||||
final int origTableRowCount = MobSnapshotTestingUtils.countMobRows(original);
|
||||
|
||||
// Take a snapshot
|
||||
final String snapshotNameAsString = "snapshot_" + localTableName;
|
||||
byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
|
||||
|
||||
SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
|
||||
snapshotNameAsString, rootDir, fs, online);
|
||||
|
||||
if (!online) {
|
||||
admin.enableTable(localTableName);
|
||||
}
|
||||
TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
|
||||
admin.cloneSnapshot(snapshotName, cloneTableName);
|
||||
|
||||
Table clonedTable = ConnectionFactory.createConnection(UTIL.getConfiguration())
|
||||
.getTable(cloneTableName);
|
||||
|
||||
try {
|
||||
final int clonedTableRowCount = MobSnapshotTestingUtils.countMobRows(clonedTable);
|
||||
|
||||
Assert.assertEquals(
|
||||
"The line counts of original and cloned tables do not match after clone. ",
|
||||
origTableRowCount, clonedTableRowCount);
|
||||
|
||||
// Attempt to add data to the test
|
||||
final String rowKey = "new-row-" + System.currentTimeMillis();
|
||||
|
||||
Put p = new Put(Bytes.toBytes(rowKey));
|
||||
p.add(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
|
||||
original.put(p);
|
||||
|
||||
// Verify that it is not present in the original table
|
||||
Assert.assertEquals("The row count of the original table was not modified by the put",
|
||||
origTableRowCount + 1, MobSnapshotTestingUtils.countMobRows(original));
|
||||
Assert.assertEquals(
|
||||
"The row count of the cloned table changed as a result of addition to the original",
|
||||
clonedTableRowCount, MobSnapshotTestingUtils.countMobRows(clonedTable));
|
||||
|
||||
p = new Put(Bytes.toBytes(rowKey));
|
||||
p.addColumn(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
|
||||
clonedTable.put(p);
|
||||
|
||||
// Verify that the new family is not in the restored table's description
|
||||
Assert.assertEquals(
|
||||
"The row count of the original table was modified by the put to the clone",
|
||||
origTableRowCount + 1, MobSnapshotTestingUtils.countMobRows(original));
|
||||
Assert.assertEquals("The row count of the cloned table was not modified by the put",
|
||||
clonedTableRowCount + 1, MobSnapshotTestingUtils.countMobRows(clonedTable));
|
||||
} finally {
|
||||
|
||||
clonedTable.close();
|
||||
}
|
||||
} finally {
|
||||
|
||||
original.close();
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Take a snapshot of a table, do a split, and verify that this only affects one table
|
||||
* @param online - Whether the table is online or not during the snapshot
|
||||
*/
|
||||
private void runTestRegionOperationsIndependent(boolean online) throws Exception {
|
||||
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
|
||||
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
|
||||
|
||||
// Create a table
|
||||
Admin admin = UTIL.getHBaseAdmin();
|
||||
final long startTime = System.currentTimeMillis();
|
||||
final TableName localTableName =
|
||||
TableName.valueOf(STRING_TABLE_NAME + startTime);
|
||||
Table original = MobSnapshotTestingUtils.createMobTable(UTIL, localTableName, TEST_FAM);
|
||||
SnapshotTestingUtils.loadData(UTIL, localTableName, 500, TEST_FAM);
|
||||
final int loadedTableCount = MobSnapshotTestingUtils.countMobRows(original);
|
||||
System.out.println("Original table has: " + loadedTableCount + " rows");
|
||||
|
||||
final String snapshotNameAsString = "snapshot_" + localTableName;
|
||||
|
||||
// Create a snapshot
|
||||
SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
|
||||
snapshotNameAsString, rootDir, fs, online);
|
||||
|
||||
if (!online) {
|
||||
admin.enableTable(localTableName);
|
||||
}
|
||||
|
||||
TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
|
||||
|
||||
// Clone the snapshot
|
||||
byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
|
||||
admin.cloneSnapshot(snapshotName, cloneTableName);
|
||||
|
||||
// Verify that region information is the same pre-split
|
||||
((HTable)original).clearRegionCache();
|
||||
List<HRegionInfo> originalTableHRegions = admin.getTableRegions(localTableName);
|
||||
|
||||
final int originalRegionCount = originalTableHRegions.size();
|
||||
final int cloneTableRegionCount = admin.getTableRegions(cloneTableName).size();
|
||||
Assert.assertEquals(
|
||||
"The number of regions in the cloned table is different than in the original table.",
|
||||
originalRegionCount, cloneTableRegionCount);
|
||||
|
||||
// Split a region on the parent table
|
||||
admin.splitRegion(originalTableHRegions.get(0).getRegionName());
|
||||
waitOnSplit(UTIL.getConnection(), original, originalRegionCount);
|
||||
|
||||
// Verify that the cloned table region is not split
|
||||
final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
|
||||
Assert.assertEquals(
|
||||
"The number of regions in the cloned table changed though none of its regions were split.",
|
||||
cloneTableRegionCount, cloneTableRegionCount2);
|
||||
}
|
||||
|
||||
/*
|
||||
* Take a snapshot of a table, add metadata, and verify that this only
|
||||
* affects one table
|
||||
* @param online - Whether the table is online or not during the snapshot
|
||||
*/
|
||||
private void runTestSnapshotMetadataChangesIndependent(boolean online) throws Exception {
|
||||
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
|
||||
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
|
||||
|
||||
// Create a table
|
||||
Admin admin = UTIL.getHBaseAdmin();
|
||||
final long startTime = System.currentTimeMillis();
|
||||
final TableName localTableName =
|
||||
TableName.valueOf(STRING_TABLE_NAME + startTime);
|
||||
Table original = MobSnapshotTestingUtils.createMobTable(UTIL, localTableName, TEST_FAM);
|
||||
SnapshotTestingUtils.loadData(UTIL, localTableName, 500, TEST_FAM);
|
||||
|
||||
final String snapshotNameAsString = "snapshot_" + localTableName;
|
||||
|
||||
// Create a snapshot
|
||||
SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
|
||||
snapshotNameAsString, rootDir, fs, online);
|
||||
|
||||
if (!online) {
|
||||
admin.enableTable(localTableName);
|
||||
}
|
||||
TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
|
||||
|
||||
// Clone the snapshot
|
||||
byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
|
||||
admin.cloneSnapshot(snapshotName, cloneTableName);
|
||||
|
||||
// Add a new column family to the original table
|
||||
byte[] TEST_FAM_2 = Bytes.toBytes("fam2");
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM_2);
|
||||
|
||||
admin.disableTable(localTableName);
|
||||
admin.addColumn(localTableName, hcd);
|
||||
|
||||
// Verify that it is not in the snapshot
|
||||
admin.enableTable(localTableName);
|
||||
|
||||
// get a description of the cloned table
|
||||
// get a list of its families
|
||||
// assert that the family is there
|
||||
HTableDescriptor originalTableDescriptor = original.getTableDescriptor();
|
||||
HTableDescriptor clonedTableDescriptor = admin.getTableDescriptor(cloneTableName);
|
||||
|
||||
Assert.assertTrue("The original family was not found. There is something wrong. ",
|
||||
originalTableDescriptor.hasFamily(TEST_FAM));
|
||||
Assert.assertTrue("The original family was not found in the clone. There is something wrong. ",
|
||||
clonedTableDescriptor.hasFamily(TEST_FAM));
|
||||
|
||||
Assert.assertTrue("The new family was not found. ",
|
||||
originalTableDescriptor.hasFamily(TEST_FAM_2));
|
||||
Assert.assertTrue("The new family was not found. ",
|
||||
!clonedTableDescriptor.hasFamily(TEST_FAM_2));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,305 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
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.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
|
||||
import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotManifestV1;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Test create/using/deleting snapshots from the client
|
||||
* <p>
|
||||
* This is an end-to-end test for the snapshot utility
|
||||
*/
|
||||
@Category({LargeTests.class, ClientTests.class})
|
||||
public class TestMobSnapshotFromClient {
|
||||
private static final Log LOG = LogFactory.getLog(TestSnapshotFromClient.class);
|
||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
private static final int NUM_RS = 2;
|
||||
private static final String STRING_TABLE_NAME = "test";
|
||||
protected static final byte[] TEST_FAM = Bytes.toBytes("fam");
|
||||
protected static final TableName TABLE_NAME =
|
||||
TableName.valueOf(STRING_TABLE_NAME);
|
||||
|
||||
/**
|
||||
* Setup the config for the cluster
|
||||
* @throws Exception on failure
|
||||
*/
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
setupConf(UTIL.getConfiguration());
|
||||
UTIL.startMiniCluster(NUM_RS);
|
||||
}
|
||||
|
||||
private static void setupConf(Configuration conf) {
|
||||
// disable the ui
|
||||
conf.setInt("hbase.regionsever.info.port", -1);
|
||||
// change the flush size to a small amount, regulating number of store files
|
||||
conf.setInt("hbase.hregion.memstore.flush.size", 25000);
|
||||
// so make sure we get a compaction when doing a load, but keep around some
|
||||
// files in the store
|
||||
conf.setInt("hbase.hstore.compaction.min", 10);
|
||||
conf.setInt("hbase.hstore.compactionThreshold", 10);
|
||||
// block writes if we get to 12 store files
|
||||
conf.setInt("hbase.hstore.blockingStoreFiles", 12);
|
||||
// Enable snapshot
|
||||
conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
|
||||
conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
ConstantSizeRegionSplitPolicy.class.getName());
|
||||
conf.setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
MobSnapshotTestingUtils.createMobTable(UTIL, TABLE_NAME, getNumReplicas(), TEST_FAM);
|
||||
}
|
||||
|
||||
protected int getNumReplicas() {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
UTIL.deleteTable(TABLE_NAME);
|
||||
SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
|
||||
SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupTest() throws Exception {
|
||||
try {
|
||||
UTIL.shutdownMiniCluster();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("failure shutting down cluster", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test snapshotting not allowed hbase:meta and -ROOT-
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testMetaTablesSnapshot() throws Exception {
|
||||
Admin admin = UTIL.getHBaseAdmin();
|
||||
byte[] snapshotName = Bytes.toBytes("metaSnapshot");
|
||||
|
||||
try {
|
||||
admin.snapshot(snapshotName, TableName.META_TABLE_NAME);
|
||||
fail("taking a snapshot of hbase:meta should not be allowed");
|
||||
} catch (IllegalArgumentException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test HBaseAdmin#deleteSnapshots(String) which deletes snapshots whose names match the parameter
|
||||
*
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testSnapshotDeletionWithRegex() throws Exception {
|
||||
Admin admin = UTIL.getHBaseAdmin();
|
||||
// make sure we don't fail on listing snapshots
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
|
||||
// put some stuff in the table
|
||||
Table table = ConnectionFactory.createConnection(UTIL.getConfiguration()).getTable(TABLE_NAME);
|
||||
UTIL.loadTable(table, TEST_FAM);
|
||||
table.close();
|
||||
|
||||
byte[] snapshot1 = Bytes.toBytes("TableSnapshot1");
|
||||
admin.snapshot(snapshot1, TABLE_NAME);
|
||||
LOG.debug("Snapshot1 completed.");
|
||||
|
||||
byte[] snapshot2 = Bytes.toBytes("TableSnapshot2");
|
||||
admin.snapshot(snapshot2, TABLE_NAME);
|
||||
LOG.debug("Snapshot2 completed.");
|
||||
|
||||
String snapshot3 = "3rdTableSnapshot";
|
||||
admin.snapshot(Bytes.toBytes(snapshot3), TABLE_NAME);
|
||||
LOG.debug(snapshot3 + " completed.");
|
||||
|
||||
// delete the first two snapshots
|
||||
admin.deleteSnapshots("TableSnapshot.*");
|
||||
List<SnapshotDescription> snapshots = admin.listSnapshots();
|
||||
assertEquals(1, snapshots.size());
|
||||
assertEquals(snapshots.get(0).getName(), snapshot3);
|
||||
|
||||
admin.deleteSnapshot(snapshot3);
|
||||
admin.close();
|
||||
}
|
||||
/**
|
||||
* Test snapshotting a table that is offline
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testOfflineTableSnapshot() throws Exception {
|
||||
Admin admin = UTIL.getHBaseAdmin();
|
||||
// make sure we don't fail on listing snapshots
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
|
||||
// put some stuff in the table
|
||||
Table table = ConnectionFactory.createConnection(UTIL.getConfiguration()).getTable(TABLE_NAME);
|
||||
UTIL.loadTable(table, TEST_FAM, false);
|
||||
|
||||
LOG.debug("FS state before disable:");
|
||||
FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
|
||||
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
|
||||
// XXX if this is flakey, might want to consider using the async version and looping as
|
||||
// disableTable can succeed and still timeout.
|
||||
admin.disableTable(TABLE_NAME);
|
||||
|
||||
LOG.debug("FS state before snapshot:");
|
||||
FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
|
||||
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
|
||||
|
||||
// take a snapshot of the disabled table
|
||||
final String SNAPSHOT_NAME = "offlineTableSnapshot";
|
||||
byte[] snapshot = Bytes.toBytes(SNAPSHOT_NAME);
|
||||
|
||||
SnapshotDescription desc = SnapshotDescription.newBuilder()
|
||||
.setType(SnapshotDescription.Type.DISABLED)
|
||||
.setTable(STRING_TABLE_NAME)
|
||||
.setName(SNAPSHOT_NAME)
|
||||
.setVersion(SnapshotManifestV1.DESCRIPTOR_VERSION)
|
||||
.build();
|
||||
admin.snapshot(desc);
|
||||
LOG.debug("Snapshot completed.");
|
||||
|
||||
// make sure we have the snapshot
|
||||
List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
|
||||
snapshot, TABLE_NAME);
|
||||
|
||||
// make sure its a valid snapshot
|
||||
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
|
||||
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
|
||||
LOG.debug("FS state after snapshot:");
|
||||
FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
|
||||
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
|
||||
|
||||
SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir,
|
||||
admin, fs);
|
||||
|
||||
admin.deleteSnapshot(snapshot);
|
||||
snapshots = admin.listSnapshots();
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
}
|
||||
|
||||
@Test (timeout=300000)
|
||||
public void testSnapshotFailsOnNonExistantTable() throws Exception {
|
||||
Admin admin = UTIL.getHBaseAdmin();
|
||||
// make sure we don't fail on listing snapshots
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
String tableName = "_not_a_table";
|
||||
|
||||
// make sure the table doesn't exist
|
||||
boolean fail = false;
|
||||
do {
|
||||
try {
|
||||
admin.getTableDescriptor(TableName.valueOf(tableName));
|
||||
fail = true;
|
||||
LOG.error("Table:" + tableName + " already exists, checking a new name");
|
||||
tableName = tableName+"!";
|
||||
} catch (TableNotFoundException e) {
|
||||
fail = false;
|
||||
}
|
||||
} while (fail);
|
||||
|
||||
// snapshot the non-existant table
|
||||
try {
|
||||
admin.snapshot("fail", TableName.valueOf(tableName));
|
||||
fail("Snapshot succeeded even though there is not table.");
|
||||
} catch (SnapshotCreationException e) {
|
||||
LOG.info("Correctly failed to snapshot a non-existant table:" + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test (timeout=300000)
|
||||
public void testOfflineTableSnapshotWithEmptyRegions() throws Exception {
|
||||
// test with an empty table with one region
|
||||
|
||||
Admin admin = UTIL.getHBaseAdmin();
|
||||
// make sure we don't fail on listing snapshots
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
|
||||
LOG.debug("FS state before disable:");
|
||||
FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
|
||||
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
|
||||
admin.disableTable(TABLE_NAME);
|
||||
|
||||
LOG.debug("FS state before snapshot:");
|
||||
FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
|
||||
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
|
||||
|
||||
// take a snapshot of the disabled table
|
||||
byte[] snapshot = Bytes.toBytes("testOfflineTableSnapshotWithEmptyRegions");
|
||||
admin.snapshot(snapshot, TABLE_NAME);
|
||||
LOG.debug("Snapshot completed.");
|
||||
|
||||
// make sure we have the snapshot
|
||||
List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
|
||||
snapshot, TABLE_NAME);
|
||||
|
||||
// make sure its a valid snapshot
|
||||
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
|
||||
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
|
||||
LOG.debug("FS state after snapshot:");
|
||||
FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
|
||||
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
|
||||
|
||||
List<byte[]> emptyCfs = Lists.newArrayList(TEST_FAM); // no file in the region
|
||||
List<byte[]> nonEmptyCfs = Lists.newArrayList();
|
||||
SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, nonEmptyCfs, emptyCfs,
|
||||
rootDir, admin, fs);
|
||||
|
||||
admin.deleteSnapshot(snapshot);
|
||||
snapshots = admin.listSnapshots();
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,87 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Assert;
|
||||
|
||||
public class MobTestUtil {
|
||||
protected static final char FIRST_CHAR = 'a';
|
||||
protected static final char LAST_CHAR = 'z';
|
||||
|
||||
protected static String generateRandomString(int demoLength) {
|
||||
String base = "abcdefghijklmnopqrstuvwxyz";
|
||||
Random random = new Random();
|
||||
StringBuffer sb = new StringBuffer();
|
||||
for (int i = 0; i < demoLength; i++) {
|
||||
int number = random.nextInt(base.length());
|
||||
sb.append(base.charAt(number));
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
protected static void writeStoreFile(final StoreFile.Writer writer, String caseName)
|
||||
throws IOException {
|
||||
writeStoreFile(writer, Bytes.toBytes(caseName), Bytes.toBytes(caseName));
|
||||
}
|
||||
|
||||
/*
|
||||
* Writes HStoreKey and ImmutableBytes data to passed writer and then closes
|
||||
* it.
|
||||
*
|
||||
* @param writer
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
private static void writeStoreFile(final StoreFile.Writer writer, byte[] fam,
|
||||
byte[] qualifier) throws IOException {
|
||||
long now = System.currentTimeMillis();
|
||||
try {
|
||||
for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) {
|
||||
for (char e = FIRST_CHAR; e <= LAST_CHAR; e++) {
|
||||
byte[] b = new byte[] { (byte) d, (byte) e };
|
||||
writer.append(new KeyValue(b, fam, qualifier, now, b));
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
writer.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare two Cells only for their row family qualifier value
|
||||
*/
|
||||
public static void assertCellEquals(Cell firstKeyValue,
|
||||
Cell secondKeyValue) {
|
||||
Assert.assertEquals(Bytes.toString(CellUtil.cloneRow(firstKeyValue)),
|
||||
Bytes.toString(CellUtil.cloneRow(secondKeyValue)));
|
||||
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(firstKeyValue)),
|
||||
Bytes.toString(CellUtil.cloneFamily(secondKeyValue)));
|
||||
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(firstKeyValue)),
|
||||
Bytes.toString(CellUtil.cloneQualifier(secondKeyValue)));
|
||||
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(firstKeyValue)),
|
||||
Bytes.toString(CellUtil.cloneValue(secondKeyValue)));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,154 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestCachedMobFile extends TestCase{
|
||||
static final Log LOG = LogFactory.getLog(TestCachedMobFile.class);
|
||||
private Configuration conf = HBaseConfiguration.create();
|
||||
private CacheConfig cacheConf = new CacheConfig(conf);
|
||||
private static final String TABLE = "tableName";
|
||||
private static final String FAMILY = "familyName";
|
||||
private static final String FAMILY1 = "familyName1";
|
||||
private static final String FAMILY2 = "familyName2";
|
||||
private static final long EXPECTED_REFERENCE_ZERO = 0;
|
||||
private static final long EXPECTED_REFERENCE_ONE = 1;
|
||||
private static final long EXPECTED_REFERENCE_TWO = 2;
|
||||
|
||||
@Test
|
||||
public void testOpenClose() throws Exception {
|
||||
String caseName = getName();
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Path testDir = FSUtils.getRootDir(conf);
|
||||
Path outputDir = new Path(new Path(testDir, TABLE),
|
||||
FAMILY);
|
||||
HFileContext meta = new HFileContextBuilder().withBlockSize(8*1024).build();
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs)
|
||||
.withOutputDir(outputDir).withFileContext(meta).build();
|
||||
MobTestUtil.writeStoreFile(writer, caseName);
|
||||
CachedMobFile cachedMobFile = CachedMobFile.create(fs, writer.getPath(), conf, cacheConf);
|
||||
Assert.assertEquals(EXPECTED_REFERENCE_ZERO, cachedMobFile.getReferenceCount());
|
||||
cachedMobFile.open();
|
||||
Assert.assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile.getReferenceCount());
|
||||
cachedMobFile.open();
|
||||
Assert.assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile.getReferenceCount());
|
||||
cachedMobFile.close();
|
||||
Assert.assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile.getReferenceCount());
|
||||
cachedMobFile.close();
|
||||
Assert.assertEquals(EXPECTED_REFERENCE_ZERO, cachedMobFile.getReferenceCount());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompare() throws Exception {
|
||||
String caseName = getName();
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Path testDir = FSUtils.getRootDir(conf);
|
||||
Path outputDir1 = new Path(new Path(testDir, TABLE),
|
||||
FAMILY1);
|
||||
HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
|
||||
StoreFile.Writer writer1 = new StoreFile.WriterBuilder(conf, cacheConf, fs)
|
||||
.withOutputDir(outputDir1).withFileContext(meta).build();
|
||||
MobTestUtil.writeStoreFile(writer1, caseName);
|
||||
CachedMobFile cachedMobFile1 = CachedMobFile.create(fs, writer1.getPath(), conf, cacheConf);
|
||||
Path outputDir2 = new Path(new Path(testDir, TABLE),
|
||||
FAMILY2);
|
||||
StoreFile.Writer writer2 = new StoreFile.WriterBuilder(conf, cacheConf, fs)
|
||||
.withOutputDir(outputDir2)
|
||||
.withFileContext(meta)
|
||||
.build();
|
||||
MobTestUtil.writeStoreFile(writer2, caseName);
|
||||
CachedMobFile cachedMobFile2 = CachedMobFile.create(fs, writer2.getPath(), conf, cacheConf);
|
||||
cachedMobFile1.access(1);
|
||||
cachedMobFile2.access(2);
|
||||
Assert.assertEquals(cachedMobFile1.compareTo(cachedMobFile2), 1);
|
||||
Assert.assertEquals(cachedMobFile2.compareTo(cachedMobFile1), -1);
|
||||
Assert.assertEquals(cachedMobFile1.compareTo(cachedMobFile1), 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadKeyValue() throws Exception {
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Path testDir = FSUtils.getRootDir(conf);
|
||||
Path outputDir = new Path(new Path(testDir, TABLE), "familyname");
|
||||
HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs)
|
||||
.withOutputDir(outputDir).withFileContext(meta).build();
|
||||
String caseName = getName();
|
||||
MobTestUtil.writeStoreFile(writer, caseName);
|
||||
CachedMobFile cachedMobFile = CachedMobFile.create(fs, writer.getPath(), conf, cacheConf);
|
||||
byte[] family = Bytes.toBytes(caseName);
|
||||
byte[] qualify = Bytes.toBytes(caseName);
|
||||
// Test the start key
|
||||
byte[] startKey = Bytes.toBytes("aa"); // The start key bytes
|
||||
KeyValue expectedKey =
|
||||
new KeyValue(startKey, family, qualify, Long.MAX_VALUE, Type.Put, startKey);
|
||||
KeyValue seekKey = expectedKey.createKeyOnly(false);
|
||||
Cell cell = cachedMobFile.readCell(seekKey, false);
|
||||
MobTestUtil.assertCellEquals(expectedKey, cell);
|
||||
|
||||
// Test the end key
|
||||
byte[] endKey = Bytes.toBytes("zz"); // The end key bytes
|
||||
expectedKey = new KeyValue(endKey, family, qualify, Long.MAX_VALUE, Type.Put, endKey);
|
||||
seekKey = expectedKey.createKeyOnly(false);
|
||||
cell = cachedMobFile.readCell(seekKey, false);
|
||||
MobTestUtil.assertCellEquals(expectedKey, cell);
|
||||
|
||||
// Test the random key
|
||||
byte[] randomKey = Bytes.toBytes(MobTestUtil.generateRandomString(2));
|
||||
expectedKey = new KeyValue(randomKey, family, qualify, Long.MAX_VALUE, Type.Put, randomKey);
|
||||
seekKey = expectedKey.createKeyOnly(false);
|
||||
cell = cachedMobFile.readCell(seekKey, false);
|
||||
MobTestUtil.assertCellEquals(expectedKey, cell);
|
||||
|
||||
// Test the key which is less than the start key
|
||||
byte[] lowerKey = Bytes.toBytes("a1"); // Smaller than "aa"
|
||||
expectedKey = new KeyValue(startKey, family, qualify, Long.MAX_VALUE, Type.Put, startKey);
|
||||
seekKey = new KeyValue(lowerKey, family, qualify, Long.MAX_VALUE, Type.Put, lowerKey);
|
||||
cell = cachedMobFile.readCell(seekKey, false);
|
||||
MobTestUtil.assertCellEquals(expectedKey, cell);
|
||||
|
||||
// Test the key which is more than the end key
|
||||
byte[] upperKey = Bytes.toBytes("z{"); // Bigger than "zz"
|
||||
seekKey = new KeyValue(upperKey, family, qualify, Long.MAX_VALUE, Type.Put, upperKey);
|
||||
cell = cachedMobFile.readCell(seekKey, false);
|
||||
Assert.assertNull(cell);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,190 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.client.*;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(LargeTests.class)
|
||||
public class TestDefaultMobStoreFlusher {
|
||||
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private final static byte [] row1 = Bytes.toBytes("row1");
|
||||
private final static byte [] row2 = Bytes.toBytes("row2");
|
||||
private final static byte [] family = Bytes.toBytes("family");
|
||||
private final static byte [] qf1 = Bytes.toBytes("qf1");
|
||||
private final static byte [] qf2 = Bytes.toBytes("qf2");
|
||||
private final static byte [] value1 = Bytes.toBytes("value1");
|
||||
private final static byte [] value2 = Bytes.toBytes("value2");
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
|
||||
TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
|
||||
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFlushNonMobFile() throws InterruptedException {
|
||||
String TN = "testFlushNonMobFile";
|
||||
TableName tn = TableName.valueOf(TN);
|
||||
Table table = null;
|
||||
HBaseAdmin admin = null;
|
||||
|
||||
try {
|
||||
HTableDescriptor desc = new HTableDescriptor(tn);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMaxVersions(4);
|
||||
desc.addFamily(hcd);
|
||||
|
||||
admin = TEST_UTIL.getHBaseAdmin();
|
||||
admin.createTable(desc);
|
||||
table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
|
||||
.getTable(TableName.valueOf(TN));
|
||||
|
||||
//Put data
|
||||
Put put0 = new Put(row1);
|
||||
put0.addColumn(family, qf1, 1, value1);
|
||||
table.put(put0);
|
||||
|
||||
//Put more data
|
||||
Put put1 = new Put(row2);
|
||||
put1.addColumn(family, qf2, 1, value2);
|
||||
table.put(put1);
|
||||
|
||||
//Flush
|
||||
admin.flush(tn);
|
||||
|
||||
Scan scan = new Scan();
|
||||
scan.addColumn(family, qf1);
|
||||
scan.setMaxVersions(4);
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
|
||||
//Compare
|
||||
Result result = scanner.next();
|
||||
int size = 0;
|
||||
while (result != null) {
|
||||
size++;
|
||||
List<Cell> cells = result.getColumnCells(family, qf1);
|
||||
// Verify the cell size
|
||||
Assert.assertEquals(1, cells.size());
|
||||
// Verify the value
|
||||
Assert.assertEquals(Bytes.toString(value1),
|
||||
Bytes.toString(CellUtil.cloneValue(cells.get(0))));
|
||||
result = scanner.next();
|
||||
}
|
||||
scanner.close();
|
||||
Assert.assertEquals(1, size);
|
||||
admin.close();
|
||||
} catch (MasterNotRunningException e1) {
|
||||
e1.printStackTrace();
|
||||
} catch (ZooKeeperConnectionException e2) {
|
||||
e2.printStackTrace();
|
||||
} catch (IOException e3) {
|
||||
e3.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFlushMobFile() throws InterruptedException {
|
||||
String TN = "testFlushMobFile";
|
||||
TableName tn = TableName.valueOf(TN);
|
||||
Table table = null;
|
||||
Admin admin = null;
|
||||
|
||||
try {
|
||||
HTableDescriptor desc = new HTableDescriptor(tn);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(3L);
|
||||
hcd.setMaxVersions(4);
|
||||
desc.addFamily(hcd);
|
||||
|
||||
Connection c = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
|
||||
admin = c.getAdmin();
|
||||
admin.createTable(desc);
|
||||
table = c.getTable(TableName.valueOf(TN));
|
||||
|
||||
//put data
|
||||
Put put0 = new Put(row1);
|
||||
put0.addColumn(family, qf1, 1, value1);
|
||||
table.put(put0);
|
||||
|
||||
//put more data
|
||||
Put put1 = new Put(row2);
|
||||
put1.addColumn(family, qf2, 1, value2);
|
||||
table.put(put1);
|
||||
|
||||
//flush
|
||||
admin.flush(tn);
|
||||
|
||||
//Scan
|
||||
Scan scan = new Scan();
|
||||
scan.addColumn(family, qf1);
|
||||
scan.setMaxVersions(4);
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
|
||||
//Compare
|
||||
Result result = scanner.next();
|
||||
int size = 0;
|
||||
while (result != null) {
|
||||
size++;
|
||||
List<Cell> cells = result.getColumnCells(family, qf1);
|
||||
// Verify the the cell size
|
||||
Assert.assertEquals(1, cells.size());
|
||||
// Verify the value
|
||||
Assert.assertEquals(Bytes.toString(value1),
|
||||
Bytes.toString(CellUtil.cloneValue(cells.get(0))));
|
||||
result = scanner.next();
|
||||
}
|
||||
scanner.close();
|
||||
Assert.assertEquals(1, size);
|
||||
admin.close();
|
||||
} catch (MasterNotRunningException e1) {
|
||||
e1.printStackTrace();
|
||||
} catch (ZooKeeperConnectionException e2) {
|
||||
e2.printStackTrace();
|
||||
} catch (IOException e3) {
|
||||
e3.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,175 @@
|
|||
/**
|
||||
* 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.mob;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.*;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
public class TestExpiredMobFileCleaner {
|
||||
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private final static TableName tableName = TableName.valueOf("TestExpiredMobFileCleaner");
|
||||
private final static String family = "family";
|
||||
private final static byte[] row1 = Bytes.toBytes("row1");
|
||||
private final static byte[] row2 = Bytes.toBytes("row2");
|
||||
private final static byte[] qf = Bytes.toBytes("qf");
|
||||
|
||||
private static BufferedMutator table;
|
||||
private static Admin admin;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
|
||||
TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
|
||||
|
||||
TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
admin.disableTable(tableName);
|
||||
admin.deleteTable(tableName);
|
||||
admin.close();
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
TEST_UTIL.getTestFileSystem().delete(TEST_UTIL.getDataTestDir(), true);
|
||||
}
|
||||
|
||||
private void init() throws Exception {
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(3L);
|
||||
hcd.setMaxVersions(4);
|
||||
desc.addFamily(hcd);
|
||||
|
||||
admin = TEST_UTIL.getHBaseAdmin();
|
||||
admin.createTable(desc);
|
||||
table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
|
||||
.getBufferedMutator(tableName);
|
||||
}
|
||||
|
||||
private void modifyColumnExpiryDays(int expireDays) throws Exception {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(3L);
|
||||
// change ttl as expire days to make some row expired
|
||||
int timeToLive = expireDays * secondsOfDay();
|
||||
hcd.setTimeToLive(timeToLive);
|
||||
|
||||
admin.modifyColumn(tableName, hcd);
|
||||
}
|
||||
|
||||
private void putKVAndFlush(BufferedMutator table, byte[] row, byte[] value, long ts)
|
||||
throws Exception {
|
||||
|
||||
Put put = new Put(row, ts);
|
||||
put.addColumn(Bytes.toBytes(family), qf, value);
|
||||
table.mutate(put);
|
||||
|
||||
table.flush();
|
||||
admin.flush(tableName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a 3 day old hfile and an 1 day old hfile then sets expiry to 2 days.
|
||||
* Verifies that the 3 day old hfile is removed but the 1 day one is still present
|
||||
* after the expiry based cleaner is run.
|
||||
*/
|
||||
@Test
|
||||
public void testCleaner() throws Exception {
|
||||
init();
|
||||
|
||||
Path mobDirPath = getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, family);
|
||||
|
||||
byte[] dummyData = makeDummyData(600);
|
||||
long ts = System.currentTimeMillis() - 3 * secondsOfDay() * 1000; // 3 days before
|
||||
putKVAndFlush(table, row1, dummyData, ts);
|
||||
FileStatus[] firstFiles = TEST_UTIL.getTestFileSystem().listStatus(mobDirPath);
|
||||
//the first mob file
|
||||
assertEquals("Before cleanup without delay 1", 1, firstFiles.length);
|
||||
String firstFile = firstFiles[0].getPath().getName();
|
||||
|
||||
ts = System.currentTimeMillis() - 1 * secondsOfDay() * 1000; // 1 day before
|
||||
putKVAndFlush(table, row2, dummyData, ts);
|
||||
FileStatus[] secondFiles = TEST_UTIL.getTestFileSystem().listStatus(mobDirPath);
|
||||
//now there are 2 mob files
|
||||
assertEquals("Before cleanup without delay 2", 2, secondFiles.length);
|
||||
String f1 = secondFiles[0].getPath().getName();
|
||||
String f2 = secondFiles[1].getPath().getName();
|
||||
String secondFile = f1.equals(firstFile) ? f2 : f1;
|
||||
|
||||
modifyColumnExpiryDays(2); // ttl = 2, make the first row expired
|
||||
|
||||
//run the cleaner
|
||||
String[] args = new String[2];
|
||||
args[0] = tableName.getNameAsString();
|
||||
args[1] = family;
|
||||
ToolRunner.run(TEST_UTIL.getConfiguration(), new ExpiredMobFileCleaner(), args);
|
||||
|
||||
FileStatus[] filesAfterClean = TEST_UTIL.getTestFileSystem().listStatus(mobDirPath);
|
||||
String lastFile = filesAfterClean[0].getPath().getName();
|
||||
//the first mob fie is removed
|
||||
assertEquals("After cleanup without delay 1", 1, filesAfterClean.length);
|
||||
assertEquals("After cleanup without delay 2", secondFile, lastFile);
|
||||
}
|
||||
|
||||
private Path getMobFamilyPath(Configuration conf, TableName tableName, String familyName) {
|
||||
Path p = new Path(MobUtils.getMobRegionPath(conf, tableName), familyName);
|
||||
return p;
|
||||
}
|
||||
|
||||
private int secondsOfDay() {
|
||||
return 24 * 3600;
|
||||
}
|
||||
|
||||
private byte[] makeDummyData(int size) {
|
||||
byte [] dummyData = new byte[size];
|
||||
new Random().nextBytes(dummyData);
|
||||
return dummyData;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,135 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.*;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
public class TestMobDataBlockEncoding {
|
||||
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private final static byte [] row1 = Bytes.toBytes("row1");
|
||||
private final static byte [] family = Bytes.toBytes("family");
|
||||
private final static byte [] qf1 = Bytes.toBytes("qualifier1");
|
||||
private final static byte [] qf2 = Bytes.toBytes("qualifier2");
|
||||
protected final byte[] qf3 = Bytes.toBytes("qualifier3");
|
||||
private static Table table;
|
||||
private static HBaseAdmin admin;
|
||||
private static HColumnDescriptor hcd;
|
||||
private static HTableDescriptor desc;
|
||||
private static Random random = new Random();
|
||||
private static long defaultThreshold = 10;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
|
||||
TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.info.port.auto", true);
|
||||
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
public void setUp(long threshold, String TN, DataBlockEncoding encoding)
|
||||
throws Exception {
|
||||
desc = new HTableDescriptor(TableName.valueOf(TN));
|
||||
hcd = new HColumnDescriptor(family);
|
||||
hcd.setMobEnabled(true);
|
||||
hcd.setMobThreshold(threshold);
|
||||
hcd.setMaxVersions(4);
|
||||
hcd.setDataBlockEncoding(encoding);
|
||||
desc.addFamily(hcd);
|
||||
admin = TEST_UTIL.getHBaseAdmin();
|
||||
admin.createTable(desc);
|
||||
table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
|
||||
.getTable(TableName.valueOf(TN));
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate the mob value.
|
||||
*
|
||||
* @param size the size of the value
|
||||
* @return the mob value generated
|
||||
*/
|
||||
private static byte[] generateMobValue(int size) {
|
||||
byte[] mobVal = new byte[size];
|
||||
random.nextBytes(mobVal);
|
||||
return mobVal;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataBlockEncoding() throws Exception {
|
||||
for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
|
||||
testDataBlockEncoding(encoding);
|
||||
}
|
||||
}
|
||||
|
||||
public void testDataBlockEncoding(DataBlockEncoding encoding) throws Exception {
|
||||
String TN = "testDataBlockEncoding" + encoding;
|
||||
setUp(defaultThreshold, TN, encoding);
|
||||
long ts1 = System.currentTimeMillis();
|
||||
long ts2 = ts1 + 1;
|
||||
long ts3 = ts1 + 2;
|
||||
byte[] value = generateMobValue((int) defaultThreshold + 1);
|
||||
|
||||
Put put1 = new Put(row1);
|
||||
put1.addColumn(family, qf1, ts3, value);
|
||||
put1.addColumn(family, qf2, ts2, value);
|
||||
put1.addColumn(family, qf3, ts1, value);
|
||||
table.put(put1);
|
||||
admin.flush(TableName.valueOf(TN));
|
||||
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(4);
|
||||
|
||||
ResultScanner results = table.getScanner(scan);
|
||||
int count = 0;
|
||||
for (Result res : results) {
|
||||
List<Cell> cells = res.listCells();
|
||||
for(Cell cell : cells) {
|
||||
// Verify the value
|
||||
Assert.assertEquals(Bytes.toString(value),
|
||||
Bytes.toString(CellUtil.cloneValue(cell)));
|
||||
count++;
|
||||
}
|
||||
}
|
||||
results.close();
|
||||
Assert.assertEquals(3, count);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,124 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestMobFile extends TestCase {
|
||||
static final Log LOG = LogFactory.getLog(TestMobFile.class);
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private Configuration conf = TEST_UTIL.getConfiguration();
|
||||
private CacheConfig cacheConf = new CacheConfig(conf);
|
||||
private final String TABLE = "tableName";
|
||||
private final String FAMILY = "familyName";
|
||||
|
||||
@Test
|
||||
public void testReadKeyValue() throws Exception {
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Path testDir = FSUtils.getRootDir(conf);
|
||||
Path outputDir = new Path(new Path(testDir, TABLE), FAMILY);
|
||||
HFileContext meta = new HFileContextBuilder().withBlockSize(8*1024).build();
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs)
|
||||
.withOutputDir(outputDir)
|
||||
.withFileContext(meta)
|
||||
.build();
|
||||
String caseName = getName();
|
||||
MobTestUtil.writeStoreFile(writer, caseName);
|
||||
|
||||
MobFile mobFile = new MobFile(new StoreFile(fs, writer.getPath(),
|
||||
conf, cacheConf, BloomType.NONE));
|
||||
byte[] family = Bytes.toBytes(caseName);
|
||||
byte[] qualify = Bytes.toBytes(caseName);
|
||||
|
||||
// Test the start key
|
||||
byte[] startKey = Bytes.toBytes("aa"); // The start key bytes
|
||||
KeyValue expectedKey =
|
||||
new KeyValue(startKey, family, qualify, Long.MAX_VALUE, Type.Put, startKey);
|
||||
KeyValue seekKey = expectedKey.createKeyOnly(false);
|
||||
Cell cell = mobFile.readCell(seekKey, false);
|
||||
MobTestUtil.assertCellEquals(expectedKey, cell);
|
||||
|
||||
// Test the end key
|
||||
byte[] endKey = Bytes.toBytes("zz"); // The end key bytes
|
||||
expectedKey = new KeyValue(endKey, family, qualify, Long.MAX_VALUE, Type.Put, endKey);
|
||||
seekKey = expectedKey.createKeyOnly(false);
|
||||
cell = mobFile.readCell(seekKey, false);
|
||||
MobTestUtil.assertCellEquals(expectedKey, cell);
|
||||
|
||||
// Test the random key
|
||||
byte[] randomKey = Bytes.toBytes(MobTestUtil.generateRandomString(2));
|
||||
expectedKey = new KeyValue(randomKey, family, qualify, Long.MAX_VALUE, Type.Put, randomKey);
|
||||
seekKey = expectedKey.createKeyOnly(false);
|
||||
cell = mobFile.readCell(seekKey, false);
|
||||
MobTestUtil.assertCellEquals(expectedKey, cell);
|
||||
|
||||
// Test the key which is less than the start key
|
||||
byte[] lowerKey = Bytes.toBytes("a1"); // Smaller than "aa"
|
||||
expectedKey = new KeyValue(startKey, family, qualify, Long.MAX_VALUE, Type.Put, startKey);
|
||||
seekKey = new KeyValue(lowerKey, family, qualify, Long.MAX_VALUE, Type.Put, lowerKey);
|
||||
cell = mobFile.readCell(seekKey, false);
|
||||
MobTestUtil.assertCellEquals(expectedKey, cell);
|
||||
|
||||
// Test the key which is more than the end key
|
||||
byte[] upperKey = Bytes.toBytes("z{"); // Bigger than "zz"
|
||||
seekKey = new KeyValue(upperKey, family, qualify, Long.MAX_VALUE, Type.Put, upperKey);
|
||||
cell = mobFile.readCell(seekKey, false);
|
||||
assertNull(cell);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetScanner() throws Exception {
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Path testDir = FSUtils.getRootDir(conf);
|
||||
Path outputDir = new Path(new Path(testDir, TABLE), FAMILY);
|
||||
HFileContext meta = new HFileContextBuilder().withBlockSize(8*1024).build();
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs)
|
||||
.withOutputDir(outputDir)
|
||||
.withFileContext(meta)
|
||||
.build();
|
||||
MobTestUtil.writeStoreFile(writer, getName());
|
||||
|
||||
MobFile mobFile = new MobFile(new StoreFile(fs, writer.getPath(),
|
||||
conf, cacheConf, BloomType.NONE));
|
||||
assertNotNull(mobFile.getScanner());
|
||||
assertTrue(mobFile.getScanner() instanceof StoreFileScanner);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,202 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Date;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.regionserver.HMobStore;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestMobFileCache extends TestCase {
|
||||
static final Log LOG = LogFactory.getLog(TestMobFileCache.class);
|
||||
private HBaseTestingUtility UTIL;
|
||||
private HRegion region;
|
||||
private Configuration conf;
|
||||
private MobCacheConfig mobCacheConf;
|
||||
private MobFileCache mobFileCache;
|
||||
private Date currentDate = new Date();
|
||||
private static final String TEST_CACHE_SIZE = "2";
|
||||
private static final int EXPECTED_CACHE_SIZE_ZERO = 0;
|
||||
private static final int EXPECTED_CACHE_SIZE_ONE = 1;
|
||||
private static final int EXPECTED_CACHE_SIZE_TWO = 2;
|
||||
private static final int EXPECTED_CACHE_SIZE_THREE = 3;
|
||||
private static final long EXPECTED_REFERENCE_ONE = 1;
|
||||
private static final long EXPECTED_REFERENCE_TWO = 2;
|
||||
|
||||
private static final String TABLE = "tableName";
|
||||
private static final String FAMILY1 = "family1";
|
||||
private static final String FAMILY2 = "family2";
|
||||
private static final String FAMILY3 = "family3";
|
||||
|
||||
private static final byte[] ROW = Bytes.toBytes("row");
|
||||
private static final byte[] ROW2 = Bytes.toBytes("row2");
|
||||
private static final byte[] VALUE = Bytes.toBytes("value");
|
||||
private static final byte[] VALUE2 = Bytes.toBytes("value2");
|
||||
private static final byte[] QF1 = Bytes.toBytes("qf1");
|
||||
private static final byte[] QF2 = Bytes.toBytes("qf2");
|
||||
private static final byte[] QF3 = Bytes.toBytes("qf3");
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
UTIL = HBaseTestingUtility.createLocalHTU();
|
||||
conf = UTIL.getConfiguration();
|
||||
HTableDescriptor htd = UTIL.createTableDescriptor("testMobFileCache");
|
||||
HColumnDescriptor hcd1 = new HColumnDescriptor(FAMILY1);
|
||||
hcd1.setMobEnabled(true);
|
||||
hcd1.setMobThreshold(0);
|
||||
HColumnDescriptor hcd2 = new HColumnDescriptor(FAMILY2);
|
||||
hcd2.setMobEnabled(true);
|
||||
hcd2.setMobThreshold(0);
|
||||
HColumnDescriptor hcd3 = new HColumnDescriptor(FAMILY3);
|
||||
hcd3.setMobEnabled(true);
|
||||
hcd3.setMobThreshold(0);
|
||||
htd.addFamily(hcd1);
|
||||
htd.addFamily(hcd2);
|
||||
htd.addFamily(hcd3);
|
||||
region = UTIL.createLocalHRegion(htd, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown() throws Exception {
|
||||
region.close();
|
||||
region.getFilesystem().delete(UTIL.getDataTestDir(), true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the mob store file.
|
||||
*/
|
||||
private Path createMobStoreFile(String family) throws IOException {
|
||||
return createMobStoreFile(HBaseConfiguration.create(), family);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the mob store file
|
||||
*/
|
||||
private Path createMobStoreFile(Configuration conf, String family) throws IOException {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMaxVersions(4);
|
||||
hcd.setMobEnabled(true);
|
||||
mobCacheConf = new MobCacheConfig(conf, hcd);
|
||||
return createMobStoreFile(hcd);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the mob store file
|
||||
*/
|
||||
private Path createMobStoreFile(HColumnDescriptor hcd)
|
||||
throws IOException {
|
||||
// Setting up a Store
|
||||
TableName tn = TableName.valueOf(TABLE);
|
||||
HTableDescriptor htd = new HTableDescriptor(tn);
|
||||
htd.addFamily(hcd);
|
||||
HMobStore mobStore = (HMobStore) region.getStore(hcd.getName());
|
||||
KeyValue key1 = new KeyValue(ROW, hcd.getName(), QF1, 1, VALUE);
|
||||
KeyValue key2 = new KeyValue(ROW, hcd.getName(), QF2, 1, VALUE);
|
||||
KeyValue key3 = new KeyValue(ROW2, hcd.getName(), QF3, 1, VALUE2);
|
||||
KeyValue[] keys = new KeyValue[] { key1, key2, key3 };
|
||||
int maxKeyCount = keys.length;
|
||||
HRegionInfo regionInfo = new HRegionInfo(tn);
|
||||
StoreFile.Writer mobWriter = mobStore.createWriterInTmp(currentDate,
|
||||
maxKeyCount, hcd.getCompactionCompression(), regionInfo.getStartKey());
|
||||
Path mobFilePath = mobWriter.getPath();
|
||||
String fileName = mobFilePath.getName();
|
||||
mobWriter.append(key1);
|
||||
mobWriter.append(key2);
|
||||
mobWriter.append(key3);
|
||||
mobWriter.close();
|
||||
String targetPathName = MobUtils.formatDate(currentDate);
|
||||
Path targetPath = new Path(mobStore.getPath(), targetPathName);
|
||||
mobStore.commitFile(mobFilePath, targetPath);
|
||||
return new Path(targetPath, fileName);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMobFileCache() throws Exception {
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
conf.set(MobConstants.MOB_FILE_CACHE_SIZE_KEY, TEST_CACHE_SIZE);
|
||||
mobFileCache = new MobFileCache(conf);
|
||||
Path file1Path = createMobStoreFile(FAMILY1);
|
||||
Path file2Path = createMobStoreFile(FAMILY2);
|
||||
Path file3Path = createMobStoreFile(FAMILY3);
|
||||
|
||||
// Before open one file by the MobFileCache
|
||||
assertEquals(EXPECTED_CACHE_SIZE_ZERO, mobFileCache.getCacheSize());
|
||||
// Open one file by the MobFileCache
|
||||
CachedMobFile cachedMobFile1 = (CachedMobFile) mobFileCache.openFile(
|
||||
fs, file1Path, mobCacheConf);
|
||||
assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
|
||||
assertNotNull(cachedMobFile1);
|
||||
assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile1.getReferenceCount());
|
||||
|
||||
// The evict is also managed by a schedule thread pool.
|
||||
// And its check period is set as 3600 seconds by default.
|
||||
// This evict should get the lock at the most time
|
||||
mobFileCache.evict(); // Cache not full, evict it
|
||||
assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
|
||||
assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile1.getReferenceCount());
|
||||
|
||||
mobFileCache.evictFile(file1Path.getName()); // Evict one file
|
||||
assertEquals(EXPECTED_CACHE_SIZE_ZERO, mobFileCache.getCacheSize());
|
||||
assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile1.getReferenceCount());
|
||||
|
||||
cachedMobFile1.close(); // Close the cached mob file
|
||||
|
||||
// Reopen three cached file
|
||||
cachedMobFile1 = (CachedMobFile) mobFileCache.openFile(
|
||||
fs, file1Path, mobCacheConf);
|
||||
assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
|
||||
CachedMobFile cachedMobFile2 = (CachedMobFile) mobFileCache.openFile(
|
||||
fs, file2Path, mobCacheConf);
|
||||
assertEquals(EXPECTED_CACHE_SIZE_TWO, mobFileCache.getCacheSize());
|
||||
CachedMobFile cachedMobFile3 = (CachedMobFile) mobFileCache.openFile(
|
||||
fs, file3Path, mobCacheConf);
|
||||
// Before the evict
|
||||
// Evict the cache, should close the first file 1
|
||||
assertEquals(EXPECTED_CACHE_SIZE_THREE, mobFileCache.getCacheSize());
|
||||
assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile1.getReferenceCount());
|
||||
assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile2.getReferenceCount());
|
||||
assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile3.getReferenceCount());
|
||||
mobFileCache.evict();
|
||||
assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
|
||||
assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile1.getReferenceCount());
|
||||
assertEquals(EXPECTED_REFERENCE_ONE, cachedMobFile2.getReferenceCount());
|
||||
assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile3.getReferenceCount());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,66 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestMobFileLink {
|
||||
|
||||
@Test
|
||||
public void testMobFilePath() throws IOException {
|
||||
TableName tableName = TableName.valueOf("testMobFilePath");
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Path rootDir = FSUtils.getRootDir(conf);
|
||||
Path tableDir = FSUtils.getTableDir(rootDir, tableName);
|
||||
Path archiveDir = FSUtils.getTableDir(HFileArchiveUtil.getArchivePath(conf), tableName);
|
||||
String fileName = "mobFile";
|
||||
String encodedRegionName = MobUtils.getMobRegionInfo(tableName).getEncodedName();
|
||||
String columnFamily = "columnFamily";
|
||||
Path regionDir = new Path(tableDir, encodedRegionName);
|
||||
Path archivedRegionDir = new Path(archiveDir, encodedRegionName);
|
||||
Path expectedMobFilePath = new Path(MobUtils.getMobFamilyPath(conf, tableName, columnFamily),
|
||||
fileName).makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
Path expectedOriginPath = new Path(new Path(regionDir, columnFamily), fileName).makeQualified(
|
||||
fs.getUri(), fs.getWorkingDirectory());
|
||||
Path expectedArchivePath = new Path(new Path(archivedRegionDir, columnFamily), fileName)
|
||||
.makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
|
||||
String hfileLinkName = tableName.getNameAsString() + "=" + encodedRegionName + "-" + fileName;
|
||||
Path hfileLinkPath = new Path(columnFamily, hfileLinkName);
|
||||
HFileLink hfileLink = HFileLink.buildFromHFileLinkPattern(conf, hfileLinkPath);
|
||||
Assert.assertEquals(expectedMobFilePath, hfileLink.getMobPath());
|
||||
Assert.assertEquals(expectedOriginPath, hfileLink.getOriginPath());
|
||||
Assert.assertEquals(expectedArchivePath, hfileLink.getArchivePath());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,79 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob;
|
||||
|
||||
import java.util.Date;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.MD5Hash;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestMobFileName extends TestCase {
|
||||
|
||||
private String uuid;
|
||||
private Date date;
|
||||
private String dateStr;
|
||||
private byte[] startKey;
|
||||
|
||||
public void setUp() {
|
||||
Random random = new Random();
|
||||
uuid = UUID.randomUUID().toString().replaceAll("-", "");
|
||||
date = new Date();
|
||||
dateStr = MobUtils.formatDate(date);
|
||||
startKey = Bytes.toBytes(random.nextInt());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHashCode() {
|
||||
assertEquals(MobFileName.create(startKey, dateStr, uuid).hashCode(),
|
||||
MobFileName.create(startKey, dateStr, uuid).hashCode());
|
||||
assertNotSame(MobFileName.create(startKey, dateStr, uuid).hashCode(),
|
||||
MobFileName.create(startKey, dateStr, uuid).hashCode());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreate() {
|
||||
MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
|
||||
assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGet() {
|
||||
MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
|
||||
assertEquals(MD5Hash.getMD5AsHex(startKey, 0, startKey.length), mobFileName.getStartKey());
|
||||
assertEquals(dateStr, mobFileName.getDate());
|
||||
assertEquals(mobFileName.getFileName(), MD5Hash.getMD5AsHex(startKey, 0, startKey.length)
|
||||
+ dateStr + uuid);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEquals() {
|
||||
MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
|
||||
assertTrue(mobFileName.equals(mobFileName));
|
||||
assertFalse(mobFileName.equals(this));
|
||||
assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid)));
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,60 @@
|
|||
/**
|
||||
*
|
||||
* 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.mob.compactions;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
|
||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestPartitionedMobCompactionRequest {
|
||||
|
||||
@Test
|
||||
public void testCompactedPartitionId() {
|
||||
String startKey1 = "startKey1";
|
||||
String startKey2 = "startKey2";
|
||||
String date1 = "date1";
|
||||
String date2 = "date2";
|
||||
CompactionPartitionId partitionId1 = new CompactionPartitionId(startKey1, date1);
|
||||
CompactionPartitionId partitionId2 = new CompactionPartitionId(startKey2, date2);
|
||||
CompactionPartitionId partitionId3 = new CompactionPartitionId(startKey1, date2);
|
||||
|
||||
Assert.assertTrue(partitionId1.equals(partitionId1));
|
||||
Assert.assertFalse(partitionId1.equals(partitionId2));
|
||||
Assert.assertFalse(partitionId1.equals(partitionId3));
|
||||
Assert.assertFalse(partitionId2.equals(partitionId3));
|
||||
|
||||
Assert.assertEquals(startKey1, partitionId1.getStartKey());
|
||||
Assert.assertEquals(date1, partitionId1.getDate());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactedPartition() {
|
||||
CompactionPartitionId partitionId = new CompactionPartitionId("startKey1", "date1");
|
||||
CompactionPartition partition = new CompactionPartition(partitionId);
|
||||
FileStatus file = new FileStatus(1, false, 1, 1024, 1, new Path("/test"));
|
||||
partition.addFile(file);
|
||||
Assert.assertEquals(file, partition.listFiles().get(0));
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue