HBASE-6052 Convert .META. and -ROOT- content to pb

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1377965 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-08-28 03:40:47 +00:00
parent f63e6a41a5
commit c9b028862a
47 changed files with 1394 additions and 2091 deletions

View File

@ -347,12 +347,15 @@ public final class HConstants {
/**
* The current version of the meta table.
* Before this the meta had HTableDescriptor serialized into the HRegionInfo;
* i.e. pre-hbase 0.92. There was no META_VERSION column in the root table
* in this case. The presence of a version and its value being zero indicates
* meta is up-to-date.
* - pre-hbase 0.92. There is no META_VERSION column in the root table
* in this case. The meta has HTableDescriptor serialized into the HRegionInfo;
* - version 0 is 0.92 and 0.94. Meta data has serialized HRegionInfo's using
* Writable serialization, and HRegionInfo's does not contain HTableDescriptors.
* - version 1 for 0.96+ keeps HRegionInfo data structures, but changes the
* byte[] serialization from Writables to Protobuf.
* See HRegionInfo.VERSION
*/
public static final short META_VERSION = 0;
public static final short META_VERSION = 1;
// Other constants

View File

@ -19,13 +19,16 @@
*/
package org.apache.hadoop.hbase;
import java.io.BufferedInputStream;
import java.io.ByteArrayInputStream;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.io.SequenceInputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -36,7 +39,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
@ -44,9 +47,9 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.JenkinsHash;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.VersionedWritable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.io.DataInputBuffer;
import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
@ -57,10 +60,30 @@ import com.google.protobuf.InvalidProtocolBufferException;
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class HRegionInfo extends VersionedWritable
implements WritableComparable<HRegionInfo> {
// VERSION == 0 when HRegionInfo had an HTableDescriptor inside it.
public static final byte VERSION_PRE_092 = 0;
public class HRegionInfo implements Comparable<HRegionInfo> {
/*
* There are two versions associated with HRegionInfo: HRegionInfo.VERSION and
* HConstants.META_VERSION. HRegionInfo.VERSION indicates the data structure's versioning
* while HConstants.META_VERSION indicates the versioning of the serialized HRIs stored in
* the META table.
*
* Pre-0.92:
* HRI.VERSION == 0 and HConstants.META_VERSION does not exist (is not stored at META table)
* HRegionInfo had an HTableDescriptor reference inside it.
* HRegionInfo is serialized as Writable to META table.
* For 0.92.x and 0.94.x:
* HRI.VERSION == 1 and HConstants.META_VERSION == 0
* HRI no longer has HTableDescriptor in it.
* HRI is serialized as Writable to META table.
* For 0.96.x:
* HRI.VERSION == 1 and HConstants.META_VERSION == 1
* HRI data structure is the same as 0.92 and 0.94
* HRI is serialized as PB to META table.
*
* Versioning of HRegionInfo is deprecated. HRegionInfo does protobuf
* serialization using RegionInfo class, which has it's own versioning.
*/
@Deprecated
public static final byte VERSION = 1;
private static final Log LOG = LogFactory.getLog(HRegionInfo.class);
@ -74,11 +97,11 @@ implements WritableComparable<HRegionInfo> {
* where,
* &lt;encodedName> is a hex version of the MD5 hash of
* &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
*
*
* The old region name format:
* &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
* For region names in the old format, the encoded name is a 32-bit
* JenkinsHash integer value (in its decimal notation, string form).
* JenkinsHash integer value (in its decimal notation, string form).
*<p>
* **NOTE**
*
@ -88,8 +111,8 @@ implements WritableComparable<HRegionInfo> {
*/
/** Separator used to demarcate the encodedName in a region name
* in the new format. See description on new format above.
*/
* in the new format. See description on new format above.
*/
private static final int ENC_SEPARATOR = '.';
public static final int MD5_HEX_LENGTH = 32;
@ -104,11 +127,11 @@ implements WritableComparable<HRegionInfo> {
if ((regionName.length >= 1)
&& (regionName[regionName.length - 1] == ENC_SEPARATOR)) {
// region name is new format. it contains the encoded name.
return true;
return true;
}
return false;
}
/**
* @param regionName
* @return the encodedName
@ -122,7 +145,7 @@ implements WritableComparable<HRegionInfo> {
regionName.length - MD5_HEX_LENGTH - 1,
MD5_HEX_LENGTH);
} else {
// old format region name. ROOT and first META region also
// old format region name. ROOT and first META region also
// use this format.EncodedName is the JenkinsHash value.
int hashVal = Math.abs(JenkinsHash.getInstance().hash(regionName,
regionName.length, 0));
@ -208,24 +231,6 @@ implements WritableComparable<HRegionInfo> {
super();
}
/**
* Used only for migration
* @param other HRegionInfoForMigration
*/
public HRegionInfo(HRegionInfo090x other) {
super();
this.endKey = other.getEndKey();
this.offLine = other.isOffline();
this.regionId = other.getRegionId();
this.regionName = other.getRegionName();
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = other.isSplit();
this.startKey = other.getStartKey();
this.hashCode = other.hashCode();
this.encodedName = other.getEncodedName();
this.tableName = other.getTableDesc().getName();
}
public HRegionInfo(final byte[] tableName) {
this(tableName, null, null);
}
@ -382,7 +387,7 @@ implements WritableComparable<HRegionInfo> {
if (md5HashBytes.length != MD5_HEX_LENGTH) {
LOG.error("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
"; Got=" + md5HashBytes.length);
"; Got=" + md5HashBytes.length);
}
// now append the bytes '.<encodedName>.' to the end
@ -391,7 +396,7 @@ implements WritableComparable<HRegionInfo> {
offset += MD5_HEX_LENGTH;
b[offset++] = ENC_SEPARATOR;
}
return b;
}
@ -502,7 +507,7 @@ implements WritableComparable<HRegionInfo> {
public byte [] getStartKey(){
return startKey;
}
/** @return the endKey */
public byte [] getEndKey(){
return endKey;
@ -702,8 +707,9 @@ implements WritableComparable<HRegionInfo> {
return this.hashCode;
}
/** @return the object version number */
@Override
/** @return the object version number
* @deprecated HRI is no longer a VersionedWritable */
@Deprecated
public byte getVersion() {
return VERSION;
}
@ -713,9 +719,8 @@ implements WritableComparable<HRegionInfo> {
* {@link #toDelimitedByteArray()}
*/
@Deprecated
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeByte(getVersion());
Bytes.writeByteArray(out, endKey);
out.writeBoolean(offLine);
out.writeLong(regionId);
@ -731,7 +736,6 @@ implements WritableComparable<HRegionInfo> {
* {@link #parseFrom(FSDataInputStream)}
*/
@Deprecated
@Override
public void readFields(DataInput in) throws IOException {
// Read the single version byte. We don't ask the super class do it
// because freaks out if its not the current classes' version. This method
@ -770,6 +774,21 @@ implements WritableComparable<HRegionInfo> {
}
}
@Deprecated
private void readFields(byte[] bytes) throws IOException {
if (bytes == null || bytes.length <= 0) {
throw new IllegalArgumentException("Can't build a writable with empty " +
"bytes array");
}
DataInputBuffer in = new DataInputBuffer();
try {
in.reset(bytes, 0, bytes.length);
this.readFields(in);
} finally {
in.close();
}
}
//
// Comparable
//
@ -817,7 +836,7 @@ implements WritableComparable<HRegionInfo> {
if (this.offLine == o.offLine)
return 0;
if (this.offLine == true) return -1;
return 1;
}
@ -919,7 +938,7 @@ implements WritableComparable<HRegionInfo> {
/**
* @param bytes A pb RegionInfo serialized with a pb magic prefix.
* @return
* @return A deserialized {@link HRegionInfo}
* @throws DeserializationException
* @see {@link #toByteArray()}
*/
@ -935,7 +954,9 @@ implements WritableComparable<HRegionInfo> {
}
} else {
try {
return (HRegionInfo)Writables.getWritable(bytes, new HRegionInfo());
HRegionInfo hri = new HRegionInfo();
hri.readFields(bytes);
return hri;
} catch (IOException e) {
throw new DeserializationException(e);
}
@ -953,6 +974,84 @@ implements WritableComparable<HRegionInfo> {
return ProtobufUtil.toDelimitedByteArray(convert());
}
/**
* Extract a HRegionInfo and ServerName from catalog table {@link Result}.
* @param r Result to pull from
* @return A pair of the {@link HRegionInfo} and the {@link ServerName}
* (or null for server address if no address set in .META.).
* @throws IOException
*/
public static Pair<HRegionInfo, ServerName> getHRegionInfoAndServerName(final Result r) {
HRegionInfo info =
getHRegionInfo(r, HConstants.REGIONINFO_QUALIFIER);
ServerName sn = getServerName(r);
return new Pair<HRegionInfo, ServerName>(info, sn);
}
/**
* Returns HRegionInfo object from the column
* HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
* table Result.
* @param data a Result object from the catalog table scan
* @return HRegionInfo or null
*/
public static HRegionInfo getHRegionInfo(Result data) {
byte [] bytes =
data.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (bytes == null) return null;
HRegionInfo info = parseFromOrNull(bytes);
if (LOG.isDebugEnabled()) {
LOG.debug("Current INFO from scan results = " + info);
}
return info;
}
/**
* Returns the daughter regions by reading the corresponding columns of the catalog table
* Result.
* @param data a Result object from the catalog table scan
* @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
* parent
*/
public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) throws IOException {
HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
return new PairOfSameType<HRegionInfo>(splitA, splitB);
}
/**
* Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* <code>qualifier</code> of the catalog table result.
* @param r a Result object from the catalog table scan
* @param qualifier Column family qualifier -- either
* {@link HConstants#SPLITA_QUALIFIER}, {@link HConstants#SPLITB_QUALIFIER} or
* {@link HConstants#REGIONINFO_QUALIFIER}.
* @return An HRegionInfo instance or null.
* @throws IOException
*/
public static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
byte [] bytes = r.getValue(HConstants.CATALOG_FAMILY, qualifier);
if (bytes == null || bytes.length <= 0) return null;
return parseFromOrNull(bytes);
}
/**
* Returns a {@link ServerName} from catalog table {@link Result}.
* @param r Result to pull from
* @return A ServerName instance or null if necessary fields not found or empty.
*/
public static ServerName getServerName(final Result r) {
byte[] value = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value == null || value.length == 0) return null;
String hostAndPort = Bytes.toString(value);
value = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
if (value == null || value.length == 0) return null;
return new ServerName(hostAndPort, Bytes.toLong(value));
}
/**
* Parses an HRegionInfo instance from the passed in stream. Presumes the HRegionInfo was
* serialized to the stream with {@link #toDelimitedByteArray()}
@ -960,23 +1059,88 @@ implements WritableComparable<HRegionInfo> {
* @return An instance of HRegionInfo.
* @throws IOException
*/
public static HRegionInfo parseFrom(final FSDataInputStream in) throws IOException {
public static HRegionInfo parseFrom(final DataInputStream in) throws IOException {
// I need to be able to move back in the stream if this is not a pb serialization so I can
// do the Writable decoding instead.
InputStream is = in.markSupported()? in: new BufferedInputStream(in);
int pblen = ProtobufUtil.lengthOfPBMagic();
is.mark(pblen);
byte [] pbuf = new byte[pblen];
int read = is.read(pbuf);
if (in.markSupported()) { //read it with mark()
in.mark(pblen);
}
int read = in.read(pbuf); //assumption: if Writable serialization, it should be longer than pblen.
if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
return convert(HBaseProtos.RegionInfo.parseDelimitedFrom(is));
return convert(HBaseProtos.RegionInfo.parseDelimitedFrom(in));
} else {
// Presume Writables. Need to reset the stream since it didn't start w/ pb.
in.reset();
HRegionInfo hri = new HRegionInfo();
hri.readFields(in);
return hri;
// Presume Writables. Need to reset the stream since it didn't start w/ pb.
if (in.markSupported()) {
in.reset();
HRegionInfo hri = new HRegionInfo();
hri.readFields(in);
return hri;
} else {
//we cannot use BufferedInputStream, it consumes more than we read from the underlying IS
ByteArrayInputStream bais = new ByteArrayInputStream(pbuf);
SequenceInputStream sis = new SequenceInputStream(bais, in); //concatenate input streams
HRegionInfo hri = new HRegionInfo();
hri.readFields(new DataInputStream(sis));
return hri;
}
}
}
/**
* Serializes given HRegionInfo's as a byte array. Use this instead of {@link #toByteArray()} when
* writing to a stream and you want to use the pb mergeDelimitedFrom (w/o the delimiter, pb reads
* to EOF which may not be what you want). {@link #parseDelimitedFrom(byte[], int, int)} can
* be used to read back the instances.
* @param infos HRegionInfo objects to serialize
* @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
* @throws IOException
* @see {@link #toByteArray()}
*/
public static byte[] toDelimitedByteArray(HRegionInfo... infos) throws IOException {
byte[][] bytes = new byte[infos.length][];
int size = 0;
for (int i = 0; i < infos.length; i++) {
bytes[i] = infos[i].toDelimitedByteArray();
size += bytes[i].length;
}
byte[] result = new byte[size];
int offset = 0;
for (byte[] b : bytes) {
System.arraycopy(b, 0, result, offset, b.length);
offset += b.length;
}
return result;
}
/**
* Parses all the HRegionInfo instances from the passed in stream until EOF. Presumes the
* HRegionInfo's were serialized to the stream with {@link #toDelimitedByteArray()}
* @param bytes serialized bytes
* @param offset the start offset into the byte[] buffer
* @param length how far we should read into the byte[] buffer
* @return All the hregioninfos that are in the byte array. Keeps reading till we hit the end.
*/
public static List<HRegionInfo> parseDelimitedFrom(final byte[] bytes, final int offset,
final int length) throws IOException {
if (bytes == null) {
throw new IllegalArgumentException("Can't build an object with empty bytes array");
}
DataInputBuffer in = new DataInputBuffer();
List<HRegionInfo> hris = new ArrayList<HRegionInfo>();
try {
in.reset(bytes, offset, length);
while (in.available() > 0) {
HRegionInfo hri = parseFrom(in);
hris.add(hri);
}
} finally {
in.close();
}
return hris;
}
}

View File

@ -32,15 +32,12 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Writables;
/**
* Writes region and assignment information to <code>.META.</code>.
* TODO: Put MetaReader and MetaEditor together; doesn't make sense having
* them distinct.
* them distinct. see HBASE-3475.
*/
@InterfaceAudience.Private
public class MetaEditor {
@ -49,11 +46,26 @@ public class MetaEditor {
// Connection.
private static final Log LOG = LogFactory.getLog(MetaEditor.class);
private static Put makePutFromRegionInfo(HRegionInfo regionInfo)
/**
* Generates and returns a Put containing the region into for the catalog table
*/
public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
throws IOException {
Put put = new Put(regionInfo.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(regionInfo));
addRegionInfo(put, regionInfo);
return put;
}
/**
* Adds split daughters to the Put
*/
public static Put addDaughtersToPut(Put put, HRegionInfo splitA, HRegionInfo splitB) {
if (splitA != null) {
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, splitA.toByteArray());
}
if (splitB != null) {
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, splitB.toByteArray());
}
return put;
}
@ -148,6 +160,39 @@ public class MetaEditor {
LOG.info("Added region " + regionInfo.getRegionNameAsString() + " to META");
}
/**
* Adds a META row for the specified new region to the given catalog table. The
* HTable is not flushed or closed.
* @param meta the HTable for META
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(HTable meta, HRegionInfo regionInfo) throws IOException {
addRegionToMeta(meta, regionInfo, null, null);
}
/**
* Adds a (single) META row for the specified new region and its daughters. Note that this does
* not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use
* {@link #offlineParentInMeta(CatalogTracker, HRegionInfo, HRegionInfo, HRegionInfo)}
* and {@link #addDaughter(CatalogTracker, HRegionInfo, ServerName)} if you want to do that.
* @param meta the HTable for META
* @param regionInfo region information
* @param splitA first split daughter of the parent regionInfo
* @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(HTable meta, HRegionInfo regionInfo,
HRegionInfo splitA, HRegionInfo splitB) throws IOException {
Put put = makePutFromRegionInfo(regionInfo);
addDaughtersToPut(put, splitA, splitB);
meta.put(put);
if (LOG.isDebugEnabled()) {
LOG.debug("Added region " + regionInfo.getRegionNameAsString() + " to META");
}
}
/**
* Adds a META row for each of the specified new regions.
* @param catalogTracker CatalogTracker
@ -181,15 +226,14 @@ public class MetaEditor {
HRegionInfo copyOfParent = new HRegionInfo(parent);
copyOfParent.setOffline(true);
copyOfParent.setSplit(true);
Put put = new Put(copyOfParent.getRegionName());
addRegionInfo(put, copyOfParent);
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
Writables.getBytes(a));
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
Writables.getBytes(b));
putToMetaTable(catalogTracker, put);
LOG.info("Offlined parent region " + parent.getRegionNameAsString() +
" in META");
HTable meta = MetaReader.getMetaHTable(catalogTracker);
try {
addRegionToMeta(meta, copyOfParent, a, b);
LOG.info("Offlined parent region " + parent.getRegionNameAsString() +
" in META");
} finally {
meta.close();
}
}
public static void addDaughter(final CatalogTracker catalogTracker,
@ -297,32 +341,10 @@ public class MetaEditor {
", from parent " + parent.getRegionNameAsString());
}
public static HRegionInfo getHRegionInfo(
Result data) throws IOException {
byte [] bytes =
data.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (bytes == null) return null;
HRegionInfo info = Writables.getHRegionInfo(bytes);
LOG.info("Current INFO from scan results = " + info);
return info;
}
/**
* Returns the daughter regions by reading from the corresponding columns of the .META. table
* Result. If the region is not a split parent region, it returns PairOfSameType(null, null).
*/
public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) throws IOException {
HRegionInfo splitA = Writables.getHRegionInfoOrNull(
data.getValue(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER));
HRegionInfo splitB = Writables.getHRegionInfoOrNull(
data.getValue(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER));
return new PairOfSameType<HRegionInfo>(splitA, splitB);
}
private static Put addRegionInfo(final Put p, final HRegionInfo hri)
throws IOException {
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
hri.toByteArray());
return p;
}

View File

@ -0,0 +1,220 @@
/**
* 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.catalog;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.DeserializationException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.catalog.MetaReader.Visitor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
/**
* A tool to migrate the data stored in ROOT and META tables to pbuf serialization.
* Supports migrating from 0.92.x and 0.94.x to 0.96.x for the catalog tables.
* @deprecated will be removed for the major release after 0.96.
*/
@Deprecated
public class MetaMigrationConvertingToPB {
private static final Log LOG = LogFactory.getLog(MetaMigrationConvertingToPB.class);
private static class ConvertToPBMetaVisitor implements Visitor {
private final MasterServices services;
private long numMigratedRows;
public ConvertToPBMetaVisitor(MasterServices services) {
this.services = services;
numMigratedRows = 0;
}
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
// Check info:regioninfo, info:splitA, and info:splitB. Make sure all
// have migrated HRegionInfos.
byte [] hriBytes = getBytes(r, HConstants.REGIONINFO_QUALIFIER);
// Presumes that an edit updating all three cells either succeeds or
// doesn't -- that we don't have case of info:regioninfo migrated but not
// info:splitA.
if (isMigrated(hriBytes)) return true;
// OK. Need to migrate this row in meta.
//This will 'migrate' the HRI from 092.x and 0.94.x to 0.96+ by reading the
//writable serialization
HRegionInfo hri = parseFrom(hriBytes);
// Now make a put to write back to meta.
Put p = MetaEditor.makePutFromRegionInfo(hri);
// Now migrate info:splitA and info:splitB if they are not null
migrateSplitIfNecessary(r, p, HConstants.SPLITA_QUALIFIER);
migrateSplitIfNecessary(r, p, HConstants.SPLITB_QUALIFIER);
MetaEditor.putToCatalogTable(this.services.getCatalogTracker(), p);
if (LOG.isDebugEnabled()) {
LOG.debug("Migrated " + Bytes.toString(p.getRow()));
}
numMigratedRows++;
return true;
}
}
static void migrateSplitIfNecessary(final Result r, final Put p, final byte [] which)
throws IOException {
byte [] hriSplitBytes = getBytes(r, which);
if (!isMigrated(hriSplitBytes)) {
//This will 'migrate' the HRI from 092.x and 0.94.x to 0.96+ by reading the
//writable serialization
HRegionInfo hri = parseFrom(hriSplitBytes);
p.add(HConstants.CATALOG_FAMILY, which, hri.toByteArray());
}
}
static HRegionInfo parseFrom(byte[] hriBytes) throws IOException {
try {
return HRegionInfo.parseFrom(hriBytes);
} catch (DeserializationException ex) {
throw new IOException(ex);
}
}
/**
* @param r Result to dig in.
* @param qualifier Qualifier to look at in the passed <code>r</code>.
* @return Bytes for an HRegionInfo or null if no bytes or empty bytes found.
*/
static byte [] getBytes(final Result r, final byte [] qualifier) {
byte [] hriBytes = r.getValue(HConstants.CATALOG_FAMILY, qualifier);
if (hriBytes == null || hriBytes.length <= 0) return null;
return hriBytes;
}
static boolean isMigrated(final byte [] hriBytes) {
if (hriBytes == null || hriBytes.length <= 0) return true;
return ProtobufUtil.isPBMagicPrefix(hriBytes);
}
/**
* Update ROOT and META to newer version, converting writable serialization to PB, if
* it is needed.
* @param services MasterServices to get a handle on master
* @return num migrated rows
* @throws IOException or RuntimeException if something goes wrong
*/
public static long updateRootAndMetaIfNecessary(final MasterServices services)
throws IOException {
if (isMetaHRIUpdated(services.getCatalogTracker())) {
LOG.info("ROOT/META already up-to date with PB serialization");
return 0;
}
LOG.info("ROOT/META has Writable serializations, migrating ROOT and META to PB serialization");
try {
long rows = updateRootAndMeta(services);
LOG.info("ROOT and META updated with PB serialization. Total rows updated: " + rows);
return rows;
} catch (IOException e) {
LOG.warn("Update ROOT/META with PB serialization failed." +
"Master startup aborted.");
throw e;
}
}
/**
* Update ROOT and META to newer version, converting writable serialization to PB
* @return num migrated rows
*/
static long updateRootAndMeta(final MasterServices masterServices)
throws IOException {
long rows = updateRoot(masterServices);
rows += updateMeta(masterServices);
return rows;
}
/**
* Update ROOT rows, converting writable serialization to PB
* @return num migrated rows
*/
static long updateRoot(final MasterServices masterServices)
throws IOException {
LOG.info("Starting update of ROOT");
ConvertToPBMetaVisitor v = new ConvertToPBMetaVisitor(masterServices);
MetaReader.fullScan(masterServices.getCatalogTracker(), v, null, true);
LOG.info("Finished update of ROOT. Total rows updated:" + v.numMigratedRows);
return v.numMigratedRows;
}
/**
* Update META rows, converting writable serialization to PB
* @return num migrated rows
*/
static long updateMeta(final MasterServices masterServices) throws IOException {
LOG.info("Starting update of META");
ConvertToPBMetaVisitor v = new ConvertToPBMetaVisitor(masterServices);
MetaReader.fullScan(masterServices.getCatalogTracker(), v);
updateRootWithMetaMigrationStatus(masterServices.getCatalogTracker());
LOG.info("Finished update of META. Total rows updated:" + v.numMigratedRows);
return v.numMigratedRows;
}
/**
* Update the version flag in -ROOT-.
* @param catalogTracker the catalog tracker
* @throws IOException
*/
static void updateRootWithMetaMigrationStatus(final CatalogTracker catalogTracker)
throws IOException {
Put p = new Put(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
p.add(HConstants.CATALOG_FAMILY, HConstants.META_VERSION_QUALIFIER,
Bytes.toBytes(HConstants.META_VERSION));
MetaEditor.putToRootTable(catalogTracker, p);
LOG.info("Updated -ROOT- meta version=" + HConstants.META_VERSION);
}
/**
* @param catalogTracker the catalog tracker
* @return True if the meta table has been migrated.
* @throws IOException
*/
static boolean isMetaHRIUpdated(final CatalogTracker catalogTracker) throws IOException {
List<Result> results = MetaReader.fullScanOfRoot(catalogTracker);
if (results == null || results.isEmpty()) {
LOG.info(".META. is not migrated");
return false;
}
// Presume only the one result because we only support one meta region.
Result r = results.get(0);
byte [] value = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.META_VERSION_QUALIFIER);
short version = value == null || value.length <= 0? -1: Bytes.toShort(value);
boolean migrated = version >= HConstants.META_VERSION;
LOG.info("Meta version=" + version + "; migrated=" + migrated);
return migrated;
}
}

View File

@ -1,276 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.catalog.MetaReader.Visitor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
/**
* Tools to help with migration of meta tables so they no longer host
* instances of HTableDescriptor.
* @deprecated Used migration from 0.90 to 0.92 so will be going away in next
* release
*/
public class MetaMigrationRemovingHTD {
private static final Log LOG = LogFactory.getLog(MetaMigrationRemovingHTD.class);
/**
* Update legacy META rows, removing HTD from HRI.
* @param masterServices
* @return List of table descriptors.
* @throws IOException
*/
public static Set<HTableDescriptor> updateMetaWithNewRegionInfo(
final MasterServices masterServices)
throws IOException {
MigratingVisitor v = new MigratingVisitor(masterServices);
MetaReader.fullScan(masterServices.getCatalogTracker(), v);
updateRootWithMetaMigrationStatus(masterServices.getCatalogTracker());
return v.htds;
}
/**
* Update the ROOT with new HRI. (HRI with no HTD)
* @param masterServices
* @return List of table descriptors
* @throws IOException
*/
static Set<HTableDescriptor> updateRootWithNewRegionInfo(
final MasterServices masterServices)
throws IOException {
MigratingVisitor v = new MigratingVisitor(masterServices);
MetaReader.fullScan(masterServices.getCatalogTracker(), v, null, true);
return v.htds;
}
/**
* Meta visitor that migrates the info:regioninfo as it visits.
*/
static class MigratingVisitor implements Visitor {
private final MasterServices services;
final Set<HTableDescriptor> htds = new HashSet<HTableDescriptor>();
MigratingVisitor(final MasterServices services) {
this.services = services;
}
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
// Check info:regioninfo, info:splitA, and info:splitB. Make sure all
// have migrated HRegionInfos... that there are no leftover 090 version
// HRegionInfos.
byte [] hriBytes = getBytes(r, HConstants.REGIONINFO_QUALIFIER);
// Presumes that an edit updating all three cells either succeeds or
// doesn't -- that we don't have case of info:regioninfo migrated but not
// info:splitA.
if (isMigrated(hriBytes)) return true;
// OK. Need to migrate this row in meta.
HRegionInfo090x hri090 = getHRegionInfo090x(hriBytes);
HTableDescriptor htd = hri090.getTableDesc();
if (htd == null) {
LOG.warn("A 090 HRI has null HTD? Continuing; " + hri090.toString());
return true;
}
if (!this.htds.contains(htd)) {
// If first time we are adding a table, then write it out to fs.
// Presumes that first region in table has THE table's schema which
// might not be too bad of a presumption since it'll be first region
// 'altered'
this.services.getMasterFileSystem().createTableDescriptor(htd);
this.htds.add(htd);
}
// This will 'migrate' the hregioninfo from 090 version to 092.
HRegionInfo hri = new HRegionInfo(hri090);
// Now make a put to write back to meta.
Put p = new Put(hri.getRegionName());
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
// Now check info:splitA and info:splitB if present. Migrate these too.
checkSplit(r, p, HConstants.SPLITA_QUALIFIER);
checkSplit(r, p, HConstants.SPLITB_QUALIFIER);
// Below we fake out putToCatalogTable
MetaEditor.putToCatalogTable(this.services.getCatalogTracker(), p);
LOG.info("Migrated " + Bytes.toString(p.getRow()));
return true;
}
}
static void checkSplit(final Result r, final Put p, final byte [] which)
throws IOException {
byte [] hriSplitBytes = getBytes(r, which);
if (!isMigrated(hriSplitBytes)) {
// This will convert the HRI from 090 to 092 HRI.
HRegionInfo hri = Writables.getHRegionInfo(hriSplitBytes);
p.add(HConstants.CATALOG_FAMILY, which, Writables.getBytes(hri));
}
}
/**
* @param r Result to dig in.
* @param qualifier Qualifier to look at in the passed <code>r</code>.
* @return Bytes for an HRegionInfo or null if no bytes or empty bytes found.
*/
static byte [] getBytes(final Result r, final byte [] qualifier) {
byte [] hriBytes = r.getValue(HConstants.CATALOG_FAMILY, qualifier);
if (hriBytes == null || hriBytes.length <= 0) return null;
return hriBytes;
}
/**
* @param r Result to look in.
* @param qualifier What to look at in the passed result.
* @return Either a 090 vintage HRegionInfo OR null if no HRegionInfo or
* the HRegionInfo is up to date and not in need of migration.
* @throws IOException
*/
static HRegionInfo090x get090HRI(final Result r, final byte [] qualifier)
throws IOException {
byte [] hriBytes = r.getValue(HConstants.CATALOG_FAMILY, qualifier);
if (hriBytes == null || hriBytes.length <= 0) return null;
if (isMigrated(hriBytes)) return null;
return getHRegionInfo090x(hriBytes);
}
static boolean isMigrated(final byte [] hriBytes) {
if (hriBytes == null || hriBytes.length <= 0) return true;
// Else, what version this HRegionInfo instance is at. The first byte
// is the version byte in a serialized HRegionInfo. If its same as our
// current HRI, then nothing to do.
if (hriBytes[0] == HRegionInfo.VERSION) return true;
if (hriBytes[0] == HRegionInfo.VERSION_PRE_092) return false;
// Unknown version. Return true that its 'migrated' but log warning.
// Should 'never' happen.
assert false: "Unexpected version; bytes=" + Bytes.toStringBinary(hriBytes);
return true;
}
/**
* Migrate root and meta to newer version. This updates the META and ROOT
* and removes the HTD from HRI.
* @param masterServices
* @throws IOException
*/
public static void migrateRootAndMeta(final MasterServices masterServices)
throws IOException {
updateRootWithNewRegionInfo(masterServices);
updateMetaWithNewRegionInfo(masterServices);
}
/**
* Update the version flag in -ROOT-.
* @param catalogTracker
* @throws IOException
*/
public static void updateRootWithMetaMigrationStatus(final CatalogTracker catalogTracker)
throws IOException {
Put p = new Put(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
MetaEditor.putToRootTable(catalogTracker, setMetaVersion(p));
LOG.info("Updated -ROOT- meta version=" + HConstants.META_VERSION);
}
static Put setMetaVersion(final Put p) {
p.add(HConstants.CATALOG_FAMILY, HConstants.META_VERSION_QUALIFIER,
Bytes.toBytes(HConstants.META_VERSION));
return p;
}
/**
* @return True if the meta table has been migrated.
* @throws IOException
*/
// Public because used in tests
public static boolean isMetaHRIUpdated(final MasterServices services)
throws IOException {
List<Result> results = MetaReader.fullScanOfRoot(services.getCatalogTracker());
if (results == null || results.isEmpty()) {
LOG.info("Not migrated");
return false;
}
// Presume only the one result because we only support on meta region.
Result r = results.get(0);
short version = getMetaVersion(r);
boolean migrated = version >= HConstants.META_VERSION;
LOG.info("Meta version=" + version + "; migrated=" + migrated);
return migrated;
}
/**
* @param r Result to look at
* @return Current meta table version or -1 if no version found.
*/
static short getMetaVersion(final Result r) {
byte [] value = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.META_VERSION_QUALIFIER);
return value == null || value.length <= 0? -1: Bytes.toShort(value);
}
/**
* @return True if migrated.
* @throws IOException
*/
public static boolean updateMetaWithNewHRI(final MasterServices services)
throws IOException {
if (isMetaHRIUpdated(services)) {
LOG.info("ROOT/Meta already up-to date with new HRI.");
return true;
}
LOG.info("Meta has HRI with HTDs. Updating meta now.");
try {
migrateRootAndMeta(services);
LOG.info("ROOT and Meta updated with new HRI.");
return true;
} catch (IOException e) {
throw new RuntimeException("Update ROOT/Meta with new HRI failed." +
"Master startup aborted.");
}
}
/**
* Get HREgionInfoForMigration serialized from bytes.
* @param bytes serialized bytes
* @return An instance of a 090 HRI or null if we failed deserialize
*/
public static HRegionInfo090x getHRegionInfo090x(final byte [] bytes) {
if (bytes == null || bytes.length == 0) return null;
HRegionInfo090x hri = null;
try {
hri = (HRegionInfo090x)Writables.getWritable(bytes, new HRegionInfo090x());
} catch (IOException ioe) {
LOG.warn("Failed deserialize as a 090 HRegionInfo); bytes=" +
Bytes.toStringBinary(bytes), ioe);
}
return hri;
}
}

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Writables;
/**
* Reads region and assignment information from <code>.META.</code>.
@ -114,7 +113,7 @@ public class MetaReader {
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
Pair<HRegionInfo, ServerName> region = parseCatalogResult(r);
Pair<HRegionInfo, ServerName> region = HRegionInfo.getHRegionInfoAndServerName(r);
if (region == null) return true;
HRegionInfo hri = region.getFirst();
if (hri == null) return true;
@ -202,7 +201,7 @@ public class MetaReader {
/**
* Callers should call close on the returned {@link HTable} instance.
* @param catalogTracker
* @param row Row we are putting
* @param row Row we are putting
* @return
* @throws IOException
*/
@ -289,59 +288,7 @@ public class MetaReader {
Get get = new Get(regionName);
get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getCatalogHTable(catalogTracker, regionName), get);
return (r == null || r.isEmpty())? null: parseCatalogResult(r);
}
/**
* Extract a {@link ServerName}
* For use on catalog table {@link Result}.
* @param r Result to pull from
* @return A ServerName instance or null if necessary fields not found or empty.
*/
public static ServerName getServerNameFromCatalogResult(final Result r) {
byte[] value = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value == null || value.length == 0) return null;
String hostAndPort = Bytes.toString(value);
value = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
if (value == null || value.length == 0) return null;
return new ServerName(hostAndPort, Bytes.toLong(value));
}
/**
* Extract a HRegionInfo and ServerName.
* For use on catalog table {@link Result}.
* @param r Result to pull from
* @return A pair of the {@link HRegionInfo} and the {@link ServerName}
* (or null for server address if no address set in .META.).
* @throws IOException
*/
public static Pair<HRegionInfo, ServerName> parseCatalogResult(final Result r)
throws IOException {
HRegionInfo info =
parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
ServerName sn = getServerNameFromCatalogResult(r);
return new Pair<HRegionInfo, ServerName>(info, sn);
}
/**
* Parse the content of the cell at {@link HConstants#CATALOG_FAMILY} and
* <code>qualifier</code> as an HRegionInfo and return it, or null.
* For use on catalog table {@link Result}.
* @param r Result instance to pull from.
* @param qualifier Column family qualifier -- either
* {@link HConstants#SPLITA_QUALIFIER}, {@link HConstants#SPLITB_QUALIFIER} or
* {@link HConstants#REGIONINFO_QUALIFIER}.
* @return An HRegionInfo instance or null.
* @throws IOException
*/
public static HRegionInfo parseHRegionInfoFromCatalogResult(final Result r,
byte [] qualifier)
throws IOException {
byte [] bytes = r.getValue(HConstants.CATALOG_FAMILY, qualifier);
if (bytes == null || bytes.length <= 0) return null;
return Writables.getHRegionInfoOrNull(bytes);
return (r == null || r.isEmpty())? null: HRegionInfo.getHRegionInfoAndServerName(r);
}
/**
@ -368,7 +315,7 @@ public class MetaReader {
@Override
public boolean visit(Result r) throws IOException {
this.current =
parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
HRegionInfo.getHRegionInfo(r, HConstants.REGIONINFO_QUALIFIER);
if (this.current == null) {
LOG.warn("No serialized HRegionInfo in " + r);
return true;
@ -522,14 +469,14 @@ public class MetaReader {
@Override
public boolean visit(Result r) throws IOException {
HRegionInfo hri =
parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
HRegionInfo.getHRegionInfo(r, HConstants.REGIONINFO_QUALIFIER);
if (hri == null) {
LOG.warn("No serialized HRegionInfo in " + r);
return true;
}
if (!isInsideTable(hri, tableName)) return false;
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
ServerName sn = getServerNameFromCatalogResult(r);
ServerName sn = HRegionInfo.getServerName(r);
// Populate this.current so available when we call #add
this.current = new Pair<HRegionInfo, ServerName>(hri, sn);
// Else call super and add this Result to the collection.
@ -563,8 +510,8 @@ public class MetaReader {
@Override
void add(Result r) {
if (r == null || r.isEmpty()) return;
ServerName sn = getServerNameFromCatalogResult(r);
if (sn != null && sn.equals(serverName)) this.results.add(r);
ServerName sn = HRegionInfo.getServerName(r);
if (sn != null && sn.equals(serverName)) this.results.add(r);
}
};
fullScan(catalogTracker, v);
@ -572,7 +519,7 @@ public class MetaReader {
if (results != null && !results.isEmpty()) {
// Convert results to Map keyed by HRI
for (Result r: results) {
Pair<HRegionInfo, ServerName> p = parseCatalogResult(r);
Pair<HRegionInfo, ServerName> p = HRegionInfo.getHRegionInfoAndServerName(r);
if (p != null && p.getFirst() != null) hris.put(p.getFirst(), r);
}
}
@ -586,7 +533,7 @@ public class MetaReader {
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
HRegionInfo hrim = MetaEditor.getHRegionInfo(r);
HRegionInfo hrim = HRegionInfo.getHRegionInfo(r);
LOG.info("fullScanMetaAndPrint.HRI Print= " + hrim);
return true;
}

View File

@ -56,20 +56,18 @@ import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
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.AdminProtos.GetRegionInfoRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
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.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
@ -77,26 +75,24 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest;
import org.apache.hadoop.hbase.client.MasterAdminKeepAliveConnection;
import org.apache.hadoop.hbase.client.MasterMonitorKeepAliveConnection;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
@ -201,7 +197,7 @@ public class HBaseAdmin implements Abortable, Closeable {
this.aborted = true;
throw new RuntimeException(why, e);
}
@Override
public boolean isAborted(){
return this.aborted;
@ -409,11 +405,7 @@ public class HBaseAdmin implements Abortable, Closeable {
MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
@Override
public boolean processRow(Result rowResult) throws IOException {
if (rowResult == null || rowResult.size() <= 0) {
return true;
}
HRegionInfo info = MetaReader.parseHRegionInfoFromCatalogResult(
rowResult, HConstants.REGIONINFO_QUALIFIER);
HRegionInfo info = HRegionInfo.getHRegionInfo(rowResult);
if (info == null) {
LOG.warn("No serialized HRegionInfo in " + rowResult);
return true;
@ -421,14 +413,10 @@ public class HBaseAdmin implements Abortable, Closeable {
if (!(Bytes.equals(info.getTableName(), desc.getName()))) {
return false;
}
String hostAndPort = null;
byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
ServerName serverName = HRegionInfo.getServerName(rowResult);
// Make sure that regions are assigned to server
if (value != null && value.length > 0) {
hostAndPort = Bytes.toString(value);
}
if (!(info.isOffline() || info.isSplit()) && hostAndPort != null) {
if (!(info.isOffline() || info.isSplit()) && serverName != null
&& serverName.getHostAndPort() != null) {
actualRegCount.incrementAndGet();
}
return true;
@ -610,7 +598,7 @@ public class HBaseAdmin implements Abortable, Closeable {
// continue
}
}
if (tableExists) {
throw new IOException("Retries exhausted, it took too long to wait"+
" for the table " + Bytes.toString(tableName) + " to be deleted.");
@ -1142,7 +1130,7 @@ public class HBaseAdmin implements Abortable, Closeable {
* servername is provided then based on the online regions in the specified
* regionserver the specified region will be closed. The master will not be
* informed of the close. Note that the regionname is the encoded regionname.
*
*
* @param encodedRegionName
* The encoded region name; i.e. the hash that makes up the region
* name suffix: e.g. if regionname is
@ -1704,17 +1692,13 @@ public class HBaseAdmin implements Abortable, Closeable {
MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
@Override
public boolean processRow(Result data) throws IOException {
if (data == null || data.size() <= 0) {
return true;
}
HRegionInfo info = MetaReader.parseHRegionInfoFromCatalogResult(
data, HConstants.REGIONINFO_QUALIFIER);
HRegionInfo info = HRegionInfo.getHRegionInfo(data);
if (info == null) {
LOG.warn("No serialized HRegionInfo in " + data);
return true;
}
if (!encodedName.equals(info.getEncodedName())) return true;
ServerName sn = MetaReader.getServerNameFromCatalogResult(data);
ServerName sn = HRegionInfo.getServerName(data);
result.set(new Pair<HRegionInfo, ServerName>(info, sn));
return false; // found the region, stop
}
@ -1887,7 +1871,7 @@ public class HBaseAdmin implements Abortable, Closeable {
* @param tableName the name of the table
* @return Ordered list of {@link HRegionInfo}.
* @throws IOException
*/
*/
public List<HRegionInfo> getTableRegions(final byte[] tableName)
throws IOException {
CatalogTracker ct = getCatalogTracker();
@ -1899,7 +1883,7 @@ public class HBaseAdmin implements Abortable, Closeable {
}
return Regions;
}
@Override
public void close() throws IOException {
if (this.connection != null) {
@ -1920,14 +1904,14 @@ public class HBaseAdmin implements Abortable, Closeable {
/**
* Roll the log writer. That is, start writing log messages to a new file.
*
*
* @param serverName
* The servername of the regionserver. A server name is made of host,
* port and startcode. This is mandatory. Here is an example:
* <code> host187.example.com,60020,1289493121758</code>
* @return If lots of logs, flush the returned regions so next time through
* we can clean logs. Returns null if nothing to flush. Names are actual
* region names as returned by {@link HRegionInfo#getEncodedName()}
* region names as returned by {@link HRegionInfo#getEncodedName()}
* @throws IOException if a remote or network exception occurs
* @throws FailedLogCloseException
*/

View File

@ -90,7 +90,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.SoftValueSortedMap;
import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
@ -851,14 +850,11 @@ public class HConnectionManager {
MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
@Override
public boolean processRow(Result row) throws IOException {
byte[] value = row.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
HRegionInfo info = Writables.getHRegionInfoOrNull(value);
HRegionInfo info = MetaScanner.getHRegionInfo(row);
if (info != null) {
if (Bytes.equals(tableName, info.getTableName())) {
value = row.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value == null) {
ServerName server = HRegionInfo.getServerName(row);
if (server == null) {
available.set(false);
return false;
}
@ -973,39 +969,30 @@ public class HConnectionManager {
MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
public boolean processRow(Result result) throws IOException {
try {
byte[] value = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
HRegionInfo regionInfo = null;
if (value != null) {
// convert the row result into the HRegionLocation we need!
regionInfo = Writables.getHRegionInfo(value);
// possible we got a region of a different table...
if (!Bytes.equals(regionInfo.getTableName(),
tableName)) {
return false; // stop scanning
}
if (regionInfo.isOffline()) {
// don't cache offline regions
return true;
}
value = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value == null) {
return true; // don't cache it
}
final String hostAndPort = Bytes.toString(value);
String hostname = Addressing.parseHostname(hostAndPort);
int port = Addressing.parsePort(hostAndPort);
value = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
// instantiate the location
HRegionLocation loc =
new HRegionLocation(regionInfo, hostname, port);
// cache this meta entry
cacheLocation(tableName, loc);
HRegionInfo regionInfo = MetaScanner.getHRegionInfo(result);
if (regionInfo == null) {
return true;
}
// possible we got a region of a different table...
if (!Bytes.equals(regionInfo.getTableName(), tableName)) {
return false; // stop scanning
}
if (regionInfo.isOffline()) {
// don't cache offline regions
return true;
}
ServerName serverName = HRegionInfo.getServerName(result);
if (serverName == null) {
return true; // don't cache it
}
// instantiate the location
HRegionLocation loc = new HRegionLocation(regionInfo, serverName.getHostname(),
serverName.getPort());
// cache this meta entry
cacheLocation(tableName, loc);
return true;
} catch (RuntimeException e) {
throw new IOException(e);
@ -1092,15 +1079,14 @@ public class HConnectionManager {
if (regionInfoRow == null) {
throw new TableNotFoundException(Bytes.toString(tableName));
}
byte [] value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (value == null || value.length == 0) {
// convert the row result into the HRegionLocation we need!
HRegionInfo regionInfo = MetaScanner.getHRegionInfo(regionInfoRow);
if (regionInfo == null) {
throw new IOException("HRegionInfo was null or empty in " +
Bytes.toString(parentTable) + ", row=" + regionInfoRow);
}
// convert the row result into the HRegionLocation we need!
HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(
value, new HRegionInfo());
// possible we got a region of a different table...
if (!Bytes.equals(regionInfo.getTableName(), tableName)) {
throw new TableNotFoundException(
@ -1119,13 +1105,8 @@ public class HConnectionManager {
regionInfo.getRegionNameAsString());
}
value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
String hostAndPort = "";
if (value != null) {
hostAndPort = Bytes.toString(value);
}
if (hostAndPort.equals("")) {
ServerName serverName = HRegionInfo.getServerName(regionInfoRow);
if (serverName == null) {
throw new NoServerForRegionException("No server address listed " +
"in " + Bytes.toString(parentTable) + " for region " +
regionInfo.getRegionNameAsString() + " containing row " +
@ -1133,9 +1114,8 @@ public class HConnectionManager {
}
// Instantiate the location
String hostname = Addressing.parseHostname(hostAndPort);
int port = Addressing.parsePort(hostAndPort);
location = new HRegionLocation(regionInfo, hostname, port);
location =
new HRegionLocation(regionInfo, serverName.getHostname(), serverName.getPort());
cacheLocation(tableName, location);
return location;
} catch (TableNotFoundException e) {

View File

@ -21,15 +21,16 @@ package org.apache.hadoop.hbase.client;
import java.io.Closeable;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Proxy;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Collections;
import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.concurrent.ExecutorService;
@ -54,6 +55,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.io.DataInputInputStream;
import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
import org.apache.hadoop.hbase.ipc.ExecRPCInvoker;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -77,10 +79,10 @@ import com.google.protobuf.ServiceException;
* <p>Used to communicate with a single HBase table.
*
* <p>This class is not thread safe for reads nor write.
*
*
* <p>In case of writes (Put, Delete), the underlying write buffer can
* be corrupted if multiple threads contend over a single HTable instance.
*
*
* <p>In case of reads, some fields used by a Scan are shared among all threads.
* The HTable implementation can either not contract to be safe in case of a Get
*
@ -107,7 +109,7 @@ import com.google.protobuf.ServiceException;
*
* <p>Note that this class implements the {@link Closeable} interface. When a
* HTable instance is no longer required, it *should* be closed in order to ensure
* that the underlying resources are promptly released. Please note that the close
* that the underlying resources are promptly released. Please note that the close
* method can throw java.io.IOException that must be handled.
*
* @see HBaseAdmin for create, drop, list, enable and disable of tables.
@ -224,7 +226,7 @@ public class HTable implements HTableInterface {
* @param pool ExecutorService to be used.
* @throws IOException if a remote or network exception occurs
*/
public HTable(final byte[] tableName, final HConnection connection,
public HTable(final byte[] tableName, final HConnection connection,
final ExecutorService pool) throws IOException {
if (pool == null || pool.isShutdown()) {
throw new IllegalArgumentException("Pool is null or shut down.");
@ -367,7 +369,7 @@ public class HTable implements HTableInterface {
throws IOException {
return connection.getRegionLocation(tableName, row, reload);
}
/**
* {@inheritDoc}
*/
@ -580,13 +582,16 @@ public class HTable implements HTableInterface {
* </pre>
* @param out {@link DataOutput} to serialize this object into.
* @throws IOException if a remote or network exception occurs
* @deprecated serializing/deserializing regioninfo's are deprecated
*/
@Deprecated
public void serializeRegionInfo(DataOutput out) throws IOException {
Map<HRegionInfo, HServerAddress> allRegions = this.getRegionsInfo();
// first, write number of regions
out.writeInt(allRegions.size());
for (Map.Entry<HRegionInfo, HServerAddress> es : allRegions.entrySet()) {
es.getKey().write(out);
byte[] hriBytes = es.getKey().toDelimitedByteArray();
out.write(hriBytes);
es.getValue().write(out);
}
}
@ -606,19 +611,27 @@ public class HTable implements HTableInterface {
* @param in {@link DataInput} object.
* @return A map of HRegionInfo with its server address.
* @throws IOException if an I/O exception occurs.
* @deprecated serializing/deserializing regioninfo's are deprecated
*/
@Deprecated
public Map<HRegionInfo, HServerAddress> deserializeRegionInfo(DataInput in)
throws IOException {
final Map<HRegionInfo, HServerAddress> allRegions =
new TreeMap<HRegionInfo, HServerAddress>();
DataInputStream is = null;
if (in instanceof DataInputStream) {
is = (DataInputStream) in;
} else {
is = new DataInputStream(DataInputInputStream.constructInputStream(in));
}
// the first integer is expected to be the size of records
int regionsCount = in.readInt();
int regionsCount = is.readInt();
for (int i = 0; i < regionsCount; ++i) {
HRegionInfo hri = new HRegionInfo();
hri.readFields(in);
HRegionInfo hri = HRegionInfo.parseFrom(is);
HServerAddress hsa = new HServerAddress();
hsa.readFields(in);
hsa.readFields(is);
allRegions.put(hri, hsa);
}
return allRegions;
@ -802,7 +815,7 @@ public class HTable implements HTableInterface {
validatePut(put);
writeBuffer.add(put);
currentWriteBufferSize += put.heapSize();
// we need to periodically see if the writebuffer is full instead of waiting until the end of the List
n++;
if (n % DOPUT_WB_CHECK == 0 && currentWriteBufferSize > writeBufferSize) {

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.util.PairOfSameType;
/**
* Scanner class that contains the <code>.META.</code> table scanning logic
@ -166,13 +166,11 @@ public class MetaScanner {
throw new TableNotFoundException("Cannot find row in .META. for table: "
+ Bytes.toString(tableName) + ", row=" + Bytes.toStringBinary(searchRow));
}
byte[] value = startRowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (value == null || value.length == 0) {
HRegionInfo regionInfo = getHRegionInfo(startRowResult);
if (regionInfo == null) {
throw new IOException("HRegionInfo was null or empty in Meta for " +
Bytes.toString(tableName) + ", row=" + Bytes.toStringBinary(searchRow));
}
HRegionInfo regionInfo = Writables.getHRegionInfo(value);
byte[] rowBefore = regionInfo.getStartKey();
startRow = HRegionInfo.createRegionName(tableName, rowBefore,
@ -239,6 +237,24 @@ public class MetaScanner {
} while (Bytes.compareTo(startRow, HConstants.LAST_ROW) != 0);
}
/**
* Returns HRegionInfo object from the column
* HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
* table Result.
* @param data a Result object from the catalog table scan
* @return HRegionInfo or null
*/
public static HRegionInfo getHRegionInfo(Result data) {
byte [] bytes =
data.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (bytes == null) return null;
HRegionInfo info = HRegionInfo.parseFromOrNull(bytes);
if (LOG.isDebugEnabled()) {
LOG.debug("Current INFO from scan results = " + info);
}
return info;
}
/**
* Lists all of the regions currently in META.
* @param conf
@ -267,13 +283,13 @@ public class MetaScanner {
if (result == null || result.isEmpty()) {
return true;
}
byte [] bytes = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (bytes == null) {
HRegionInfo regionInfo = getHRegionInfo(result);
if (regionInfo == null) {
LOG.warn("Null REGIONINFO_QUALIFIER: " + result);
return true;
}
HRegionInfo regionInfo = Writables.getHRegionInfo(bytes);
// If region offline AND we are not to include offlined regions, return.
if (regionInfo.isOffline() && !offlined) return true;
regions.add(regionInfo);
@ -299,25 +315,11 @@ public class MetaScanner {
MetaScannerVisitor visitor = new TableMetaScannerVisitor(conf, tablename) {
@Override
public boolean processRowInternal(Result rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfo(
rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
String hostAndPort = null;
if (value != null && value.length > 0) {
hostAndPort = Bytes.toString(value);
}
value = rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
long startcode = -1L;
if (value != null && value.length > 0) startcode = Bytes.toLong(value);
HRegionInfo info = getHRegionInfo(rowResult);
ServerName serverName = HRegionInfo.getServerName(rowResult);
if (!(info.isOffline() || info.isSplit())) {
ServerName sn = null;
if (hostAndPort != null && hostAndPort.length() > 0) {
sn = new ServerName(hostAndPort, startcode);
}
regions.put(new UnmodifyableHRegionInfo(info), sn);
regions.put(new UnmodifyableHRegionInfo(info), serverName);
}
return true;
}
@ -389,9 +391,7 @@ public class MetaScanner {
@Override
public boolean processRow(Result rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfoOrNull(
rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
HRegionInfo info = getHRegionInfo(rowResult);
if (info == null) {
return true;
}
@ -405,10 +405,9 @@ public class MetaScanner {
* seen by this scanner as well, so we block until they are added to the META table. Even
* though we are waiting for META entries, ACID semantics in HBase indicates that this
* scanner might not see the new rows. So we manually query the daughter rows */
HRegionInfo splitA = Writables.getHRegionInfo(rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER));
HRegionInfo splitB = Writables.getHRegionInfo(rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER));
PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(rowResult);
HRegionInfo splitA = daughters.getFirst();
HRegionInfo splitB = daughters.getSecond();
HTable metaTable = getMetaTable();
long start = System.currentTimeMillis();
@ -446,8 +445,7 @@ public class MetaScanner {
while (System.currentTimeMillis() - start < timeout) {
Get get = new Get(regionName);
Result result = metaTable.get(get);
HRegionInfo info = Writables.getHRegionInfoOrNull(
result.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER));
HRegionInfo info = getHRegionInfo(result);
if (info != null) {
return result;
}
@ -478,8 +476,7 @@ public class MetaScanner {
@Override
public final boolean processRow(Result rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfoOrNull(
rowResult.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER));
HRegionInfo info = getHRegionInfo(rowResult);
if (info == null) {
return true;
}

View File

@ -94,6 +94,8 @@ public class Result implements Writable, WritableWithSize {
/**
* Instantiate a Result with the specified array of KeyValues.
* <br><strong>Note:</strong> You must ensure that the keyvalues
* are already sorted
* @param kvs array of KeyValues
*/
public Result(KeyValue [] kvs) {
@ -104,6 +106,8 @@ public class Result implements Writable, WritableWithSize {
/**
* Instantiate a Result with the specified List of KeyValues.
* <br><strong>Note:</strong> You must ensure that the keyvalues
* are already sorted
* @param kvs List of KeyValues
*/
public Result(List<KeyValue> kvs) {

View File

@ -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.io;
import java.io.DataInput;
import java.io.IOException;
import java.io.InputStream;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* An InputStream that wraps a DataInput.
* @see DataOutputOutputStream
*/
@InterfaceAudience.Private
public class DataInputInputStream extends InputStream {
private DataInput in;
/**
* Construct an InputStream from the given DataInput. If 'in'
* is already an InputStream, simply returns it. Otherwise, wraps
* it in an InputStream.
* @param in the DataInput to wrap
* @return an InputStream instance that reads from 'in'
*/
public static InputStream constructInputStream(DataInput in) {
if (in instanceof InputStream) {
return (InputStream)in;
} else {
return new DataInputInputStream(in);
}
}
public DataInputInputStream(DataInput in) {
this.in = in;
}
@Override
public int read() throws IOException {
return in.readUnsignedByte();
}
}

View File

@ -73,7 +73,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.KeyLocker;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKTable;
@ -149,14 +148,14 @@ public class AssignmentManager extends ZooKeeperListener {
* Contains the server which need to update timer, these servers will be
* handled by {@link TimerUpdater}
*/
private final ConcurrentSkipListSet<ServerName> serversInUpdatingTimer =
private final ConcurrentSkipListSet<ServerName> serversInUpdatingTimer =
new ConcurrentSkipListSet<ServerName>();
private final ExecutorService executorService;
//Thread pool executor service for timeout monitor
private java.util.concurrent.ExecutorService threadPoolExecutorService;
private List<EventType> ignoreStatesRSOffline = Arrays.asList(new EventType[]{
EventType.RS_ZK_REGION_FAILED_OPEN, EventType.RS_ZK_REGION_CLOSED });
@ -166,8 +165,8 @@ public class AssignmentManager extends ZooKeeperListener {
*/
private volatile boolean failover = false;
// Set holding all the regions which got processed while RIT was not
// populated during master failover.
// Set holding all the regions which got processed while RIT was not
// populated during master failover.
private Map<String, HRegionInfo> failoverProcessedRegions =
new HashMap<String, HRegionInfo>();
@ -184,7 +183,7 @@ public class AssignmentManager extends ZooKeeperListener {
* @param catalogTracker
* @param service
* @throws KeeperException
* @throws IOException
* @throws IOException
*/
public AssignmentManager(Server master, ServerManager serverManager,
CatalogTracker catalogTracker, final LoadBalancer balancer,
@ -244,8 +243,8 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Add a regionPlan for the specified region.
* @param encodedName
* @param plan
* @param encodedName
* @param plan
*/
public void addPlan(String encodedName, RegionPlan plan) {
synchronized (regionPlans) {
@ -344,7 +343,7 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Process all regions that are in transition in zookeeper and also
* processes the list of dead servers by scanning the META.
* processes the list of dead servers by scanning the META.
* Used by master joining an cluster. If we figure this is a clean cluster
* startup, will assign all user regions.
* @param deadServers
@ -441,7 +440,7 @@ public class AssignmentManager extends ZooKeeperListener {
* up in zookeeper.
* @param encodedRegionName Region to process failover for.
* @param regionInfo If null we'll go get it from meta table.
* @param deadServers Can be null
* @param deadServers Can be null
* @return True if we processed <code>regionInfo</code> as a RIT.
* @throws KeeperException
* @throws IOException
@ -461,7 +460,7 @@ public class AssignmentManager extends ZooKeeperListener {
}
HRegionInfo hri = regionInfo;
if (hri == null) {
if ((hri = getHRegionInfo(rt.getRegionName())) == null) return false;
if ((hri = getHRegionInfo(rt.getRegionName())) == null) return false;
}
processRegionsInTransition(rt, hri, deadServers, stat.getVersion());
return true;
@ -582,7 +581,7 @@ public class AssignmentManager extends ZooKeeperListener {
lock.unlock();
}
}
/**
* Put the region <code>hri</code> into an offline state up in zk.
@ -728,7 +727,7 @@ public class AssignmentManager extends ZooKeeperListener {
byte [] payload = rt.getPayload();
List<HRegionInfo> daughters = null;
try {
daughters = Writables.getHRegionInfos(payload, 0, payload.length);
daughters = HRegionInfo.parseDelimitedFrom(payload, 0, payload.length);
} catch (IOException e) {
LOG.error("Dropped split! Failed reading split payload for " +
prettyPrintedRegionName);
@ -795,7 +794,7 @@ public class AssignmentManager extends ZooKeeperListener {
this.executorService.submit(new ClosedRegionHandler(master,
this, regionState.getRegion()));
break;
case RS_ZK_REGION_FAILED_OPEN:
hri = checkIfInFailover(regionState, encodedName, regionName);
if (hri != null) {
@ -869,7 +868,7 @@ public class AssignmentManager extends ZooKeeperListener {
this.executorService.submit(
new OpenedRegionHandler(master, this, regionState.getRegion(), sn, expectedVersion));
break;
default:
throw new IllegalStateException("Received event is not valid.");
}
@ -901,7 +900,7 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Gets the HRegionInfo from the META table
* @param regionName
* @return HRegionInfo hri for the region
* @return HRegionInfo hri for the region
*/
private HRegionInfo getHRegionInfo(final byte [] regionName) {
Pair<HRegionInfo, ServerName> p = null;
@ -1234,7 +1233,7 @@ public class AssignmentManager extends ZooKeeperListener {
*/
public void assign(HRegionInfo region, boolean setOfflineInZK,
boolean forceNewPlan, boolean hijack) {
// If hijack is true do not call disableRegionIfInRIT as
// If hijack is true do not call disableRegionIfInRIT as
// we have not yet moved the znode to OFFLINE state.
if (!hijack && isDisabledorDisablingRegionInRIT(region)) {
return;
@ -1285,7 +1284,7 @@ public class AssignmentManager extends ZooKeeperListener {
destination));
}
this.addPlans(plans);
// Presumption is that only this thread will be updating the state at this
// time; i.e. handlers on backend won't be trying to set it to OPEN, etc.
AtomicInteger counter = new AtomicInteger(0);
@ -1366,7 +1365,7 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Bulk assign regions to available servers if any with retry, else assign
* region singly.
*
*
* @param regions all regions to assign
* @param servers all available servers
*/
@ -1660,7 +1659,7 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Set region as OFFLINED up in zookeeper
*
*
* @param state
* @param hijack
* - true if needs to be hijacked and reassigned, false otherwise.
@ -1670,7 +1669,7 @@ public class AssignmentManager extends ZooKeeperListener {
int setOfflineInZooKeeper(final RegionState state,
boolean hijack) {
// In case of reassignment the current state in memory need not be
// OFFLINE.
// OFFLINE.
if (!hijack && !state.isClosed() && !state.isOffline()) {
String msg = "Unexpected state : " + state + " .. Cannot transit it to OFFLINE.";
this.master.abort(msg, new IllegalStateException(msg));
@ -1679,13 +1678,13 @@ public class AssignmentManager extends ZooKeeperListener {
boolean allowZNodeCreation = false;
// Under reassignment if the current state is PENDING_OPEN
// or OPENING then refresh the in-memory state to PENDING_OPEN. This is
// important because if the region was in
// important because if the region was in
// RS_OPENING state for a long time the master will try to force the znode
// to OFFLINE state meanwhile the RS could have opened the corresponding
// region and the state in znode will be RS_ZK_REGION_OPENED.
// For all other cases we can change the in-memory state to OFFLINE.
if (hijack &&
(state.getState().equals(RegionState.State.PENDING_OPEN) ||
(state.getState().equals(RegionState.State.PENDING_OPEN) ||
state.getState().equals(RegionState.State.OPENING))) {
regionStates.updateRegionState(state.getRegion(),
RegionState.State.PENDING_OPEN);
@ -1698,7 +1697,7 @@ public class AssignmentManager extends ZooKeeperListener {
int versionOfOfflineNode = -1;
try {
// get the version after setting the znode to OFFLINE
versionOfOfflineNode = ZKAssign.createOrForceNodeOffline(master.getZooKeeper(),
versionOfOfflineNode = ZKAssign.createOrForceNodeOffline(master.getZooKeeper(),
state.getRegion(), this.master.getServerName(),
hijack, allowZNodeCreation);
if (versionOfOfflineNode == -1) {
@ -1734,7 +1733,7 @@ public class AssignmentManager extends ZooKeeperListener {
} catch (KeeperException e) {
if (e instanceof NodeExistsException) {
LOG.warn("Node for " + state.getRegion() + " already exists");
} else {
} else {
master.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
}
return false;
@ -1918,7 +1917,7 @@ public class AssignmentManager extends ZooKeeperListener {
return;
}
} catch (KeeperException ee) {
Exception e = ee;
Exception e = ee;
if (e instanceof NodeExistsException) {
// Handle race between master initiated close and regionserver
// orchestrated splitting. See if existing node is in a
@ -1949,8 +1948,8 @@ public class AssignmentManager extends ZooKeeperListener {
}
state = regionStates.updateRegionState(region, RegionState.State.PENDING_CLOSE);
} else if (force && (state.isPendingClose() || state.isClosing())) {
LOG.debug("Attempting to unassign region " + region.getRegionNameAsString() +
" which is already " + state.getState() +
LOG.debug("Attempting to unassign region " + region.getRegionNameAsString() +
" which is already " + state.getState() +
" but forcing to send a CLOSE RPC again ");
state.updateTimestampToNow();
} else {
@ -2015,9 +2014,9 @@ public class AssignmentManager extends ZooKeeperListener {
public void unassign(HRegionInfo region, boolean force){
unassign(region, force, null);
}
/**
*
*
* @param region regioninfo of znode to be deleted.
*/
public void deleteClosingOrClosedNode(HRegionInfo region) {
@ -2048,7 +2047,7 @@ public class AssignmentManager extends ZooKeeperListener {
* @param path
* @return True if znode is in SPLIT or SPLITTING state.
* @throws KeeperException Can happen if the znode went away in meantime.
* @throws DeserializationException
* @throws DeserializationException
*/
private boolean isSplitOrSplitting(final String path)
throws KeeperException, DeserializationException {
@ -2118,7 +2117,7 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Assigns all user regions to online servers. Use round-robin assignment.
*
*
* @param regions
* @throws IOException
* @throws InterruptedException
@ -2177,7 +2176,7 @@ public class AssignmentManager extends ZooKeeperListener {
public void assignAllUserRegions() throws IOException, InterruptedException {
// Skip assignment for regions of tables in DISABLING state because during clean cluster startup
// no RS is alive and regions map also doesn't have any information about the regions.
// See HBASE-6281.
// See HBASE-6281.
Set<String> disablingAndDisabledTables = new HashSet<String>(this.disablingTables);
disablingAndDisabledTables.addAll(this.zkTable.getDisabledTables());
// Scan META for all user regions, skipping any disabled tables
@ -2409,7 +2408,7 @@ public class AssignmentManager extends ZooKeeperListener {
for (Result result : results) {
boolean disabled = false;
boolean disablingOrEnabling = false;
Pair<HRegionInfo, ServerName> region = MetaReader.parseCatalogResult(result);
Pair<HRegionInfo, ServerName> region = HRegionInfo.getHRegionInfoAndServerName(result);
if (region == null) continue;
HRegionInfo regionInfo = region.getFirst();
ServerName regionLocation = region.getSecond();
@ -2499,14 +2498,14 @@ public class AssignmentManager extends ZooKeeperListener {
} else if (checkIfRegionsBelongsToEnabling(regionInfo)) {
enablingTables.add(disablingTableName);
return true;
}
}
return false;
}
/**
* Recover the tables that were not fully moved to DISABLED state. These
* tables are in DISABLING state when the master restarted/switched.
*
*
* @param disablingTables
* @return
* @throws KeeperException
@ -2536,7 +2535,7 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Recover the tables that are not fully moved to ENABLED state. These tables
* are in ENABLING state when the master restarted/switched
*
*
* @param enablingTables
* @param isWatcherCreated
* @throws KeeperException
@ -2583,10 +2582,10 @@ public class AssignmentManager extends ZooKeeperListener {
* Processes list of dead servers from result of META scan and regions in RIT
* <p>
* This is used for failover to recover the lost regions that belonged to
* RegionServers which failed while there was no active master or regions
* RegionServers which failed while there was no active master or regions
* that were in RIT.
* <p>
*
*
* @param deadServers
* The list of dead servers which failed while there was no active
* master. Can be null.
@ -2612,7 +2611,7 @@ public class AssignmentManager extends ZooKeeperListener {
if (deadServers == null) return;
Set<ServerName> actualDeadServers = this.serverManager.getDeadServers();
for (Map.Entry<ServerName, List<Pair<HRegionInfo, Result>>> deadServer: deadServers.entrySet()) {
// skip regions of dead servers because SSH will process regions during rs expiration.
// skip regions of dead servers because SSH will process regions during rs expiration.
// see HBASE-5916
if (actualDeadServers.contains(deadServer.getKey())) {
for (Pair<HRegionInfo, Result> deadRegion : deadServer.getValue()) {
@ -2881,17 +2880,17 @@ public class AssignmentManager extends ZooKeeperListener {
"expire, send RPC again");
invokeUnassign(regionInfo);
break;
case SPLIT:
case SPLITTING:
break;
default:
throw new IllegalStateException("Received event is not valid.");
}
}
}
private void processOpeningState(HRegionInfo regionInfo) {
LOG.info("Region has been OPENING for too " + "long, reassigning region="
+ regionInfo.getRegionNameAsString());
@ -2940,7 +2939,7 @@ public class AssignmentManager extends ZooKeeperListener {
public boolean isCarryingMeta(ServerName serverName) {
return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
}
/**
* Check if the shutdown server carries the specific region.
* We have a bunch of places that store region location
@ -3044,10 +3043,10 @@ public class AssignmentManager extends ZooKeeperListener {
this.timeoutMonitor.interrupt();
this.timerUpdater.interrupt();
}
/**
* Check whether the RegionServer is online.
* @param serverName
* @param serverName
* @return True if online.
*/
public boolean isServerOnline(ServerName serverName) {

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Chore;
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.Server;
@ -50,7 +49,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.util.PairOfSameType;
/**
* A janitor for the catalog tables. Scans the <code>.META.</code> catalog
@ -124,7 +123,7 @@ class CatalogJanitor extends Chore {
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
count.incrementAndGet();
HRegionInfo info = getHRegionInfo(r);
HRegionInfo info = HRegionInfo.getHRegionInfo(r);
if (info == null) return true; // Keep scanning
if (info.isSplitParent()) splitParents.put(info, r);
// Returning true means "keep scanning"
@ -160,11 +159,10 @@ class CatalogJanitor extends Chore {
cleanParent(e.getKey(), e.getValue())) {
cleaned++;
} else {
// We could not clean the parent, so its daughters should not be cleaned either(HBASE-6160)
parentNotCleaned.add(
getDaughterRegionInfo(e.getValue(), HConstants.SPLITA_QUALIFIER).getEncodedName());
parentNotCleaned.add(
getDaughterRegionInfo(e.getValue(), HConstants.SPLITB_QUALIFIER).getEncodedName());
// We could not clean the parent, so it's daughters should not be cleaned either (HBASE-6160)
PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(e.getValue());
parentNotCleaned.add(daughters.getFirst().getEncodedName());
parentNotCleaned.add(daughters.getSecond().getEncodedName());
}
}
if (cleaned != 0) {
@ -215,24 +213,6 @@ class CatalogJanitor extends Chore {
}
}
/**
* Get HRegionInfo from passed Map of row values.
* @param result Map to do lookup in.
* @return Null if not found (and logs fact that expected COL_REGIONINFO
* was missing) else deserialized {@link HRegionInfo}
* @throws IOException
*/
static HRegionInfo getHRegionInfo(final Result result)
throws IOException {
byte [] bytes =
result.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (bytes == null) {
LOG.warn("REGIONINFO_QUALIFIER is empty in " + result);
return null;
}
return Writables.getHRegionInfo(bytes);
}
/**
* If daughters no longer hold reference to the parents, delete the parent.
* @param parent HRegionInfo of split offlined parent
@ -246,12 +226,11 @@ class CatalogJanitor extends Chore {
throws IOException {
boolean result = false;
// Run checks on each daughter split.
HRegionInfo a_region = getDaughterRegionInfo(rowContent, HConstants.SPLITA_QUALIFIER);
HRegionInfo b_region = getDaughterRegionInfo(rowContent, HConstants.SPLITB_QUALIFIER);
PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(rowContent);
Pair<Boolean, Boolean> a =
checkDaughterInFs(parent, a_region, HConstants.SPLITA_QUALIFIER);
checkDaughterInFs(parent, daughters.getFirst());
Pair<Boolean, Boolean> b =
checkDaughterInFs(parent, b_region, HConstants.SPLITB_QUALIFIER);
checkDaughterInFs(parent, daughters.getSecond());
if (hasNoReferences(a) && hasNoReferences(b)) {
LOG.debug("Deleting region " + parent.getRegionNameAsString() +
" because daughter splits no longer hold references");
@ -283,21 +262,6 @@ class CatalogJanitor extends Chore {
return !p.getFirst() || !p.getSecond();
}
/**
* Get daughter HRegionInfo out of parent info:splitA/info:splitB columns.
* @param result
* @param which Whether "info:splitA" or "info:splitB" column
* @return Deserialized content of the info:splitA or info:splitB as a
* HRegionInfo
* @throws IOException
*/
private HRegionInfo getDaughterRegionInfo(final Result result,
final byte [] which)
throws IOException {
byte [] bytes = result.getValue(HConstants.CATALOG_FAMILY, which);
return Writables.getHRegionInfoOrNull(bytes);
}
/**
* Remove mention of daughters from parent row.
* @param parent
@ -311,27 +275,24 @@ class CatalogJanitor extends Chore {
/**
* Checks if a daughter region -- either splitA or splitB -- still holds
* references to parent.
* @param parent Parent region name.
* @param split Which column family.
* @param qualifier Which of the daughters to look at, splitA or splitB.
* @param parent Parent region
* @param daughter Daughter region
* @return A pair where the first boolean says whether or not the daughter
* region directory exists in the filesystem and then the second boolean says
* whether the daughter has references to the parent.
* @throws IOException
*/
Pair<Boolean, Boolean> checkDaughterInFs(final HRegionInfo parent,
final HRegionInfo split,
final byte [] qualifier)
Pair<Boolean, Boolean> checkDaughterInFs(final HRegionInfo parent, final HRegionInfo daughter)
throws IOException {
boolean references = false;
boolean exists = false;
if (split == null) {
if (daughter == null) {
return new Pair<Boolean, Boolean>(Boolean.FALSE, Boolean.FALSE);
}
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = new Path(rootdir, split.getTableNameAsString());
Path regiondir = new Path(tabledir, split.getEncodedName());
Path tabledir = new Path(rootdir, daughter.getTableNameAsString());
Path regiondir = new Path(tabledir, daughter.getEncodedName());
exists = fs.exists(regiondir);
if (!exists) {
LOG.warn("Daughter regiondir does not exist: " + regiondir.toString());
@ -340,7 +301,7 @@ class CatalogJanitor extends Chore {
HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTableName());
for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
Path p = HStore.getStoreHomedir(tabledir, split.getEncodedName(),
Path p = HStore.getStoreHomedir(tabledir, daughter.getEncodedName(),
family.getName());
if (!fs.exists(p)) continue;
// Look for reference files. Call listStatus with anonymous instance of PathFilter.

View File

@ -80,9 +80,6 @@ import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.master.metrics.MasterMetricsWrapperImpl;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.ipc.ProtocolSignature;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
@ -99,9 +96,12 @@ import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TableEventHandler;
import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
import org.apache.hadoop.hbase.master.metrics.MasterMetricsWrapperImpl;
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
@ -252,7 +252,7 @@ Server {
private CatalogTracker catalogTracker;
// Cluster status zk tracker and local setter
private ClusterStatusTracker clusterStatusTracker;
// buffer for "fatal error" notices from region servers
// in the cluster. This is only used for assisting
// operations/debugging.
@ -399,7 +399,7 @@ Server {
"(Also watching cluster state node)");
Thread.sleep(c.getInt("zookeeper.session.timeout", 180 * 1000));
}
}
MasterMetrics getMetrics() {
@ -441,7 +441,7 @@ Server {
}
} catch (Throwable t) {
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
if (t instanceof NoClassDefFoundError &&
if (t instanceof NoClassDefFoundError &&
t.getMessage().contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) {
// improved error message for this special case
abort("HBase is having a problem with its Hadoop jars. You may need to "
@ -453,7 +453,7 @@ Server {
}
} finally {
startupStatus.cleanup();
stopChores();
// Wait for all the remaining region servers to report in IFF we were
// running a cluster shutdown AND we were NOT aborting.
@ -475,7 +475,7 @@ Server {
/**
* Try becoming active master.
* @param startupStatus
* @param startupStatus
* @return True if we could successfully become the active master.
* @throws InterruptedException
*/
@ -581,7 +581,7 @@ Server {
/**
* Finish initialization of HMaster after becoming the primary master.
*
*
* <ol>
* <li>Initialize master components - file system manager, server manager,
* assignment manager, region server tracker, catalog tracker, etc</li>
@ -593,9 +593,9 @@ Server {
* <li>Ensure assignment of root and meta regions<li>
* <li>Handle either fresh cluster start or master failover</li>
* </ol>
*
*
* @param masterRecovery
*
*
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
@ -636,7 +636,7 @@ Server {
// initialize master side coprocessors before we start handling requests
status.setStatus("Initializing master coprocessors");
this.cpHost = new MasterCoprocessorHost(this, this.conf);
// start up all service threads.
status.setStatus("Initializing master service threads");
startServiceThreads();
@ -665,12 +665,12 @@ Server {
if (!assignRootAndMeta(status)) return;
enableServerShutdownHandler();
// Update meta with new HRI if required. i.e migrate all HRI with HTD to
// HRI with out HTD in meta and update the status in ROOT. This must happen
// Update meta with new PB serialization if required. i.e migrate all HRI
// to PB serialization in meta and update the status in ROOT. This must happen
// before we assign all user regions or else the assignment will fail.
// TODO: Remove this when we do 0.94.
org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD.
updateMetaWithNewHRI(this);
// TODO: Remove this after 0.96, when we do 0.98.
org.apache.hadoop.hbase.catalog.MetaMigrationConvertingToPB
.updateRootAndMetaIfNecessary(this);
this.balancer.setMasterServices(this);
// Fixup assignment manager status
@ -683,7 +683,7 @@ Server {
status.setStatus("Fixing up missing daughters");
fixupDaughters(status);
if (!masterRecovery) {
if (!masterRecovery) {
// Start balancer and meta catalog janitor after meta and regions have
// been assigned.
status.setStatus("Starting balancer and catalog janitor");
@ -699,8 +699,8 @@ Server {
// removing dead server with same hostname and port of rs which is trying to check in before
// master initialization. See HBASE-5916.
this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
if (!masterRecovery) {
if (!masterRecovery) {
if (this.cpHost != null) {
// don't let cp initialization errors kill the master
try {
@ -711,7 +711,7 @@ Server {
}
}
}
/**
* Useful for testing purpose also where we have
* master restart scenarios.
@ -879,8 +879,7 @@ Server {
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
HRegionInfo info =
MetaReader.parseHRegionInfoFromCatalogResult(
r, HConstants.REGIONINFO_QUALIFIER);
HRegionInfo.getHRegionInfo(r);
if (info == null) return true; // Keep scanning
if (info.isOffline() && info.isSplit()) {
offlineSplitParents.put(info, r);
@ -992,7 +991,7 @@ Server {
* need to install an unexpected exception handler.
*/
void startServiceThreads() throws IOException{
// Start the executor service pools
this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
conf.getInt("hbase.master.executor.openregion.threads", 5));
@ -1002,7 +1001,7 @@ Server {
conf.getInt("hbase.master.executor.serverops.threads", 3));
this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
conf.getInt("hbase.master.executor.serverops.threads", 5));
// We depend on there being only one instance of this executor running
// at a time. To do concurrency, would need fencing of enable/disable of
// tables.
@ -1033,7 +1032,7 @@ Server {
this.infoServer.setAttribute(MASTER, this);
this.infoServer.start();
}
// Start allowing requests to happen.
this.rpcServer.openServer();
this.rpcServerOpen = true;
@ -1118,7 +1117,7 @@ Server {
/**
* @return Get remote side's InetAddress
* @throws UnknownHostException
* @throws UnknownHostException
*/
InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
throws UnknownHostException {
@ -1330,11 +1329,11 @@ Server {
newValue = this.cpHost.preBalanceSwitch(newValue);
}
if (mode == BalanceSwitchMode.SYNC) {
synchronized (this.balancer) {
synchronized (this.balancer) {
this.balanceSwitch = newValue;
}
} else {
this.balanceSwitch = newValue;
this.balanceSwitch = newValue;
}
LOG.info("BalanceSwitch=" + newValue);
if (this.cpHost != null) {
@ -1536,7 +1535,7 @@ Server {
* @return Pair indicating the number of regions updated Pair.getFirst is the
* regions that are yet to be updated Pair.getSecond is the total number
* of regions of the table
* @throws IOException
* @throws IOException
*/
@Override
public GetSchemaAlterStatusResponse getSchemaAlterStatus(
@ -1686,7 +1685,7 @@ Server {
if (data == null || data.size() <= 0) {
return true;
}
Pair<HRegionInfo, ServerName> pair = MetaReader.parseCatalogResult(data);
Pair<HRegionInfo, ServerName> pair = HRegionInfo.getHRegionInfoAndServerName(data);
if (pair == null) {
return false;
}
@ -2034,13 +2033,13 @@ Server {
public boolean isAborted() {
return this.abort;
}
void checkInitialized() throws PleaseHoldException {
if (!this.initialized) {
throw new PleaseHoldException("Master is initializing");
}
}
/**
* Report whether this master is currently the active master or not.
* If not active master, we are parked on ZK waiting to become active.

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.zookeeper.KeeperException;
@ -90,12 +91,12 @@ public class ServerShutdownHandler extends EventHandler {
}
/**
* Before assign the ROOT region, ensure it haven't
* Before assign the ROOT region, ensure it haven't
* been assigned by other place
* <p>
* Under some scenarios, the ROOT region can be opened twice, so it seemed online
* in two regionserver at the same time.
* If the ROOT region has been assigned, so the operation can be canceled.
* If the ROOT region has been assigned, so the operation can be canceled.
* @throws InterruptedException
* @throws IOException
* @throws KeeperException
@ -145,7 +146,7 @@ public class ServerShutdownHandler extends EventHandler {
}
}
}
/**
* @return True if the server we are processing was carrying <code>-ROOT-</code>
*/
@ -417,9 +418,10 @@ public class ServerShutdownHandler extends EventHandler {
final AssignmentManager assignmentManager,
final CatalogTracker catalogTracker)
throws IOException {
int fixedA = fixupDaughter(result, HConstants.SPLITA_QUALIFIER,
PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(result);
int fixedA = fixupDaughter(result, daughters.getFirst(),
assignmentManager, catalogTracker);
int fixedB = fixupDaughter(result, HConstants.SPLITB_QUALIFIER,
int fixedB = fixupDaughter(result, daughters.getSecond(),
assignmentManager, catalogTracker);
return fixedA + fixedB;
}
@ -431,12 +433,10 @@ public class ServerShutdownHandler extends EventHandler {
* @return 1 if the daughter is missing and fixed. Otherwise 0
* @throws IOException
*/
static int fixupDaughter(final Result result, final byte [] qualifier,
static int fixupDaughter(final Result result, HRegionInfo daughter,
final AssignmentManager assignmentManager,
final CatalogTracker catalogTracker)
throws IOException {
HRegionInfo daughter =
MetaReader.parseHRegionInfoFromCatalogResult(result, qualifier);
if (daughter == null) return 0;
if (isDaughterMissing(catalogTracker, daughter)) {
LOG.info("Fixup; missing daughter " + daughter.getRegionNameAsString());
@ -460,7 +460,7 @@ public class ServerShutdownHandler extends EventHandler {
* Daughter could have been split over on regionserver before a run of the
* catalogJanitor had chance to clear reference from parent.
* @param daughter Daughter region to search for.
* @throws IOException
* @throws IOException
*/
private static boolean isDaughterMissing(final CatalogTracker catalogTracker,
final HRegionInfo daughter) throws IOException {
@ -498,7 +498,7 @@ public class ServerShutdownHandler extends EventHandler {
@Override
public boolean visit(Result r) throws IOException {
HRegionInfo hri =
MetaReader.parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
HRegionInfo.getHRegionInfo(r);
if (hri == null) {
LOG.warn("No serialized HRegionInfo in " + r);
return true;

View File

@ -1,682 +0,0 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.migration;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JenkinsHash;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.hadoop.io.VersionedWritable;
import org.apache.hadoop.io.WritableComparable;
/**
* HRegion information.
* Contains HRegion id, start and end keys, a reference to this
* HRegions' table descriptor, etc.
*/
@InterfaceAudience.Private
public class HRegionInfo090x extends VersionedWritable implements
WritableComparable<HRegionInfo090x>{
private static final byte VERSION = 0;
private static final Log LOG = LogFactory.getLog(HRegionInfo090x.class);
/**
* The new format for a region name contains its encodedName at the end.
* The encoded name also serves as the directory name for the region
* in the filesystem.
*
* New region name format:
* &lt;tablename>,,&lt;startkey>,&lt;regionIdTimestamp>.&lt;encodedName>.
* where,
* &lt;encodedName> is a hex version of the MD5 hash of
* &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
*
* The old region name format:
* &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
* For region names in the old format, the encoded name is a 32-bit
* JenkinsHash integer value (in its decimal notation, string form).
*<p>
* **NOTE**
*
* ROOT, the first META region, and regions created by an older
* version of HBase (0.20 or prior) will continue to use the
* old region name format.
*/
/** Separator used to demarcate the encodedName in a region name
* in the new format. See description on new format above.
*/
private static final int ENC_SEPARATOR = '.';
public static final int MD5_HEX_LENGTH = 32;
/**
* Does region name contain its encoded name?
* @param regionName region name
* @return boolean indicating if this a new format region
* name which contains its encoded name.
*/
private static boolean hasEncodedName(final byte[] regionName) {
// check if region name ends in ENC_SEPARATOR
if ((regionName.length >= 1)
&& (regionName[regionName.length - 1] == ENC_SEPARATOR)) {
// region name is new format. it contains the encoded name.
return true;
}
return false;
}
/**
* @param regionName
* @return the encodedName
*/
public static String encodeRegionName(final byte [] regionName) {
String encodedName;
if (hasEncodedName(regionName)) {
// region is in new format:
// <tableName>,<startKey>,<regionIdTimeStamp>/encodedName/
encodedName = Bytes.toString(regionName,
regionName.length - MD5_HEX_LENGTH - 1,
MD5_HEX_LENGTH);
} else {
// old format region name. ROOT and first META region also
// use this format.EncodedName is the JenkinsHash value.
int hashVal = Math.abs(JenkinsHash.getInstance().hash(regionName,
regionName.length, 0));
encodedName = String.valueOf(hashVal);
}
return encodedName;
}
/**
* Use logging.
* @param encodedRegionName The encoded regionname.
* @return <code>-ROOT-</code> if passed <code>70236052</code> or
* <code>.META.</code> if passed </code>1028785192</code> else returns
* <code>encodedRegionName</code>
*/
public static String prettyPrint(final String encodedRegionName) {
if (encodedRegionName.equals("70236052")) {
return encodedRegionName + "/-ROOT-";
} else if (encodedRegionName.equals("1028785192")) {
return encodedRegionName + "/.META.";
}
return encodedRegionName;
}
/** delimiter used between portions of a region name */
public static final int DELIMITER = ',';
/** HRegionInfo for root region */
public static final HRegionInfo090x ROOT_REGIONINFO =
new HRegionInfo090x(0L, HTableDescriptor.ROOT_TABLEDESC);
/** HRegionInfo for first meta region */
public static final HRegionInfo090x FIRST_META_REGIONINFO =
new HRegionInfo090x(1L, HTableDescriptor.META_TABLEDESC);
private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
// This flag is in the parent of a split while the parent is still referenced
// by daughter regions. We USED to set this flag when we disabled a table
// but now table state is kept up in zookeeper as of 0.90.0 HBase.
private boolean offLine = false;
private long regionId = -1;
private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
private String regionNameStr = "";
private boolean split = false;
private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
protected HTableDescriptor tableDesc = null;
private int hashCode = -1;
//TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
public static final String NO_HASH = null;
private volatile String encodedName = NO_HASH;
private byte [] encodedNameAsBytes = null;
private void setHashCode() {
int result = Arrays.hashCode(this.regionName);
result ^= this.regionId;
result ^= Arrays.hashCode(this.startKey);
result ^= Arrays.hashCode(this.endKey);
result ^= Boolean.valueOf(this.offLine).hashCode();
result ^= this.tableDesc.hashCode();
this.hashCode = result;
}
/**
* Private constructor used constructing HRegionInfo for the catalog root and
* first meta regions
*/
private HRegionInfo090x(long regionId, HTableDescriptor tableDesc) {
super();
this.regionId = regionId;
this.tableDesc = tableDesc;
// Note: Root & First Meta regions names are still in old format
this.regionName = createRegionName(tableDesc.getName(), null,
regionId, false);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
setHashCode();
}
/** Default constructor - creates empty object */
public HRegionInfo090x() {
super();
this.tableDesc = new HTableDescriptor();
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param tableDesc the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @throws IllegalArgumentException
*/
public HRegionInfo090x(final HTableDescriptor tableDesc, final byte[] startKey,
final byte[] endKey)
throws IllegalArgumentException {
this(tableDesc, startKey, endKey, false);
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param tableDesc the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @param split true if this region has split and we have daughter regions
* regions that may or may not hold references to this region.
* @throws IllegalArgumentException
*/
public HRegionInfo090x(HTableDescriptor tableDesc, final byte[] startKey,
final byte[] endKey, final boolean split)
throws IllegalArgumentException {
this(tableDesc, startKey, endKey, split, System.currentTimeMillis());
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param tableDesc the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @param split true if this region has split and we have daughter regions
* regions that may or may not hold references to this region.
* @param regionid Region id to use.
* @throws IllegalArgumentException
*/
public HRegionInfo090x(HTableDescriptor tableDesc, final byte[] startKey,
final byte[] endKey, final boolean split, final long regionid)
throws IllegalArgumentException {
super();
if (tableDesc == null) {
throw new IllegalArgumentException("tableDesc cannot be null");
}
this.offLine = false;
this.regionId = regionid;
this.regionName = createRegionName(tableDesc.getName(), startKey, regionId, true);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = split;
this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
this.startKey = startKey == null?
HConstants.EMPTY_START_ROW: startKey.clone();
this.tableDesc = tableDesc;
setHashCode();
}
/**
* Costruct a copy of another HRegionInfo
*
* @param other
*/
public HRegionInfo090x(HRegionInfo090x other) {
super();
this.endKey = other.getEndKey();
this.offLine = other.isOffline();
this.regionId = other.getRegionId();
this.regionName = other.getRegionName();
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = other.isSplit();
this.startKey = other.getStartKey();
this.tableDesc = other.getTableDesc();
this.hashCode = other.hashCode();
this.encodedName = other.getEncodedName();
}
/**
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param regionid Region id (Usually timestamp from when region was created).
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey and id
*/
public static byte [] createRegionName(final byte [] tableName,
final byte [] startKey, final long regionid, boolean newFormat) {
return createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
}
/**
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param id Region id (Usually timestamp from when region was created).
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey and id
*/
public static byte [] createRegionName(final byte [] tableName,
final byte [] startKey, final String id, boolean newFormat) {
return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
}
/**
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param id Region id (Usually timestamp from when region was created).
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey and id
*/
public static byte [] createRegionName(final byte [] tableName,
final byte [] startKey, final byte [] id, boolean newFormat) {
byte [] b = new byte [tableName.length + 2 + id.length +
(startKey == null? 0: startKey.length) +
(newFormat ? (MD5_HEX_LENGTH + 2) : 0)];
int offset = tableName.length;
System.arraycopy(tableName, 0, b, 0, offset);
b[offset++] = DELIMITER;
if (startKey != null && startKey.length > 0) {
System.arraycopy(startKey, 0, b, offset, startKey.length);
offset += startKey.length;
}
b[offset++] = DELIMITER;
System.arraycopy(id, 0, b, offset, id.length);
offset += id.length;
if (newFormat) {
//
// Encoded name should be built into the region name.
//
// Use the region name thus far (namely, <tablename>,<startKey>,<id>)
// to compute a MD5 hash to be used as the encoded name, and append
// it to the byte buffer.
//
String md5Hash = MD5Hash.getMD5AsHex(b, 0, offset);
byte [] md5HashBytes = Bytes.toBytes(md5Hash);
if (md5HashBytes.length != MD5_HEX_LENGTH) {
LOG.error("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
"; Got=" + md5HashBytes.length);
}
// now append the bytes '.<encodedName>.' to the end
b[offset++] = ENC_SEPARATOR;
System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
offset += MD5_HEX_LENGTH;
b[offset++] = ENC_SEPARATOR;
}
return b;
}
/**
* Gets the table name from the specified region name.
* @param regionName
* @return Table name.
*/
public static byte [] getTableName(byte [] regionName) {
int offset = -1;
for (int i = 0; i < regionName.length; i++) {
if (regionName[i] == DELIMITER) {
offset = i;
break;
}
}
byte [] tableName = new byte[offset];
System.arraycopy(regionName, 0, tableName, 0, offset);
return tableName;
}
/**
* Separate elements of a regionName.
* @param regionName
* @return Array of byte[] containing tableName, startKey and id
* @throws IOException
*/
public static byte [][] parseRegionName(final byte [] regionName)
throws IOException {
int offset = -1;
for (int i = 0; i < regionName.length; i++) {
if (regionName[i] == DELIMITER) {
offset = i;
break;
}
}
if(offset == -1) throw new IOException("Invalid regionName format");
byte [] tableName = new byte[offset];
System.arraycopy(regionName, 0, tableName, 0, offset);
offset = -1;
for (int i = regionName.length - 1; i > 0; i--) {
if(regionName[i] == DELIMITER) {
offset = i;
break;
}
}
if(offset == -1) throw new IOException("Invalid regionName format");
byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
if(offset != tableName.length + 1) {
startKey = new byte[offset - tableName.length - 1];
System.arraycopy(regionName, tableName.length + 1, startKey, 0,
offset - tableName.length - 1);
}
byte [] id = new byte[regionName.length - offset - 1];
System.arraycopy(regionName, offset + 1, id, 0,
regionName.length - offset - 1);
byte [][] elements = new byte[3][];
elements[0] = tableName;
elements[1] = startKey;
elements[2] = id;
return elements;
}
/** @return the regionId */
public long getRegionId(){
return regionId;
}
/**
* @return the regionName as an array of bytes.
* @see #getRegionNameAsString()
*/
public byte [] getRegionName(){
return regionName;
}
/**
* @return Region name as a String for use in logging, etc.
*/
public String getRegionNameAsString() {
if (hasEncodedName(this.regionName)) {
// new format region names already have their encoded name.
return this.regionNameStr;
}
// old format. regionNameStr doesn't have the region name.
//
//
return this.regionNameStr + "." + this.getEncodedName();
}
/** @return the encoded region name */
public synchronized String getEncodedName() {
if (this.encodedName == NO_HASH) {
this.encodedName = encodeRegionName(this.regionName);
}
return this.encodedName;
}
public synchronized byte [] getEncodedNameAsBytes() {
if (this.encodedNameAsBytes == null) {
this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
}
return this.encodedNameAsBytes;
}
/** @return the startKey */
public byte [] getStartKey(){
return startKey;
}
/** @return the endKey */
public byte [] getEndKey(){
return endKey;
}
/**
* Returns true if the given inclusive range of rows is fully contained
* by this region. For example, if the region is foo,a,g and this is
* passed ["b","c"] or ["a","c"] it will return true, but if this is passed
* ["b","z"] it will return false.
* @throws IllegalArgumentException if the range passed is invalid (ie end < start)
*/
public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
throw new IllegalArgumentException(
"Invalid range: " + Bytes.toStringBinary(rangeStartKey) +
" > " + Bytes.toStringBinary(rangeEndKey));
}
boolean firstKeyInRange = Bytes.compareTo(rangeStartKey, startKey) >= 0;
boolean lastKeyInRange =
Bytes.compareTo(rangeEndKey, endKey) < 0 ||
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY);
return firstKeyInRange && lastKeyInRange;
}
/**
* Return true if the given row falls in this region.
*/
public boolean containsRow(byte[] row) {
return Bytes.compareTo(row, startKey) >= 0 &&
(Bytes.compareTo(row, endKey) < 0 ||
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
}
/** @return the tableDesc */
public HTableDescriptor getTableDesc(){
return tableDesc;
}
/**
* @param newDesc new table descriptor to use
*/
public void setTableDesc(HTableDescriptor newDesc) {
this.tableDesc = newDesc;
}
/** @return true if this is the root region */
public boolean isRootRegion() {
return this.tableDesc.isRootRegion();
}
/** @return true if this region is from a table that is a meta table,
* either <code>.META.</code> or <code>-ROOT-</code>
*/
public boolean isMetaTable() {
return this.tableDesc.isMetaTable();
}
/** @return true if this region is a meta region */
public boolean isMetaRegion() {
return this.tableDesc.isMetaRegion();
}
/**
* @return True if has been split and has daughters.
*/
public boolean isSplit() {
return this.split;
}
/**
* @param split set split status
*/
public void setSplit(boolean split) {
this.split = split;
}
/**
* @return True if this region is offline.
*/
public boolean isOffline() {
return this.offLine;
}
/**
* The parent of a region split is offline while split daughters hold
* references to the parent. Offlined regions are closed.
* @param offLine Set online/offline status.
*/
public void setOffline(boolean offLine) {
this.offLine = offLine;
}
/**
* @return True if this is a split parent region.
*/
public boolean isSplitParent() {
if (!isSplit()) return false;
if (!isOffline()) {
LOG.warn("Region is split but NOT offline: " + getRegionNameAsString());
}
return true;
}
/**
* @see java.lang.Object#toString()
*/
@Override
public String toString() {
return "REGION => {" + HConstants.NAME + " => '" +
this.regionNameStr +
"', STARTKEY => '" +
Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" +
Bytes.toStringBinary(this.endKey) +
"', ENCODED => " + getEncodedName() + "," +
(isOffline()? " OFFLINE => true,": "") +
(isSplit()? " SPLIT => true,": "") +
" TABLE => {" + this.tableDesc.toString() + "}";
}
/**
* @see java.lang.Object#equals(java.lang.Object)
*/
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null) {
return false;
}
if (!(o instanceof HRegionInfo090x)) {
return false;
}
return this.compareTo((HRegionInfo090x)o) == 0;
}
/**
* @see java.lang.Object#hashCode()
*/
@Override
public int hashCode() {
return this.hashCode;
}
/** @return the object version number */
@Override
public byte getVersion() {
return VERSION;
}
//
// Writable
//
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
Bytes.writeByteArray(out, endKey);
out.writeBoolean(offLine);
out.writeLong(regionId);
Bytes.writeByteArray(out, regionName);
out.writeBoolean(split);
Bytes.writeByteArray(out, startKey);
tableDesc.write(out);
out.writeInt(hashCode);
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
this.endKey = Bytes.readByteArray(in);
this.offLine = in.readBoolean();
this.regionId = in.readLong();
this.regionName = Bytes.readByteArray(in);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = in.readBoolean();
this.startKey = Bytes.readByteArray(in);
try {
this.tableDesc.readFields(in);
} catch(EOFException eofe) {
throw new IOException("HTD not found in input buffer");
}
this.hashCode = in.readInt();
}
//
// Comparable
//
public int compareTo(HRegionInfo090x o) {
if (o == null) {
return 1;
}
// Are regions of same table?
int result = Bytes.compareTo(this.tableDesc.getName(), o.tableDesc.getName());
if (result != 0) {
return result;
}
// Compare start keys.
result = Bytes.compareTo(this.startKey, o.startKey);
if (result != 0) {
return result;
}
// Compare end keys.
return Bytes.compareTo(this.endKey, o.endKey);
}
/**
* @return Comparator to use comparing {@link org.apache.hadoop.hbase.KeyValue}s.
*/
public KVComparator getComparator() {
return isRootRegion()? KeyValue.ROOT_COMPARATOR: isMetaRegion()?
KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
}
}

View File

@ -130,7 +130,6 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HashedBytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.util.StringUtils;
@ -371,7 +370,7 @@ public class HRegion implements HeapSize { // , Writable{
this.threadWakeFrequency = 0L;
this.coprocessorHost = null;
this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
this.opMetrics = new OperationMetrics();
}
@ -434,7 +433,7 @@ public class HRegion implements HeapSize { // , Writable{
setHTableSpecificConf();
this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
this.opMetrics = new OperationMetrics(conf, this.regionInfo);
/*
@ -785,7 +784,7 @@ public class HRegion implements HeapSize { // , Writable{
/**
* @param hri
* @return Content of the file we write out to the filesystem under a region
* @throws IOException
* @throws IOException
*/
private static byte [] getDotRegionInfoFileContent(final HRegionInfo hri) throws IOException {
return hri.toDelimitedByteArray();
@ -2031,7 +2030,7 @@ public class HRegion implements HeapSize { // , Writable{
try {
if (!initialized) {
this.writeRequestsCount.increment();
this.writeRequestsCount.increment();
doPreMutationHook(batchOp);
initialized = true;
}
@ -2095,9 +2094,9 @@ public class HRegion implements HeapSize { // , Writable{
boolean deletesCfSetConsistent = true;
//The set of columnFamilies first seen for Delete.
Set<byte[]> deletesCfSet = null;
WALEdit walEdit = new WALEdit();
long startTimeMs = EnvironmentEdgeManager.currentTimeMillis();
@ -2352,7 +2351,7 @@ public class HRegion implements HeapSize { // , Writable{
// do after lock
final long netTimeMs = EnvironmentEdgeManager.currentTimeMillis() - startTimeMs;
// See if the column families were consistent through the whole thing.
// if they were then keep them. If they were not then pass a null.
// null will be treated as unknown.
@ -2627,7 +2626,7 @@ public class HRegion implements HeapSize { // , Writable{
// do after lock
final long after = EnvironmentEdgeManager.currentTimeMillis();
this.opMetrics.updatePutMetrics(familyMap.keySet(), after - now);
if (flush) {
// Request a cache flush. Do it outside update lock.
@ -3788,7 +3787,7 @@ public class HRegion implements HeapSize { // , Writable{
/**
* Convenience method creating new HRegions. Used by createTable.
* The {@link HLog} for the created region needs to be closed
* The {@link HLog} for the created region needs to be closed
* explicitly, if it is not null.
* Use {@link HRegion#getLog()} to get access.
*
@ -3981,7 +3980,7 @@ public class HRegion implements HeapSize { // , Writable{
final List<KeyValue> edits = new ArrayList<KeyValue>(2);
edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER, now,
Writables.getBytes(r.getRegionInfo())));
r.getRegionInfo().toByteArray()));
// Set into the root table the version of the meta table.
edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
HConstants.META_VERSION_QUALIFIER, now,
@ -4756,16 +4755,16 @@ public class HRegion implements HeapSize { // , Writable{
} finally {
closeRegionOperation();
}
long after = EnvironmentEdgeManager.currentTimeMillis();
this.opMetrics.updateAppendMetrics(append.getFamilyMap().keySet(), after - before);
this.opMetrics.updateAppendMetrics(append.getFamilyMap().keySet(), after - before);
if (flush) {
// Request a cache flush. Do it outside update lock.
requestFlush();
}
return append.isReturnResults() ? new Result(allKVs) : null;
}
@ -4878,10 +4877,10 @@ public class HRegion implements HeapSize { // , Writable{
} finally {
closeRegionOperation();
}
long after = EnvironmentEdgeManager.currentTimeMillis();
this.opMetrics.updateIncrementMetrics(increment.getFamilyMap().keySet(), after - before);
this.opMetrics.updateIncrementMetrics(increment.getFamilyMap().keySet(), after - before);
if (flush) {
// Request a cache flush. Do it outside update lock.
requestFlush();
@ -4981,7 +4980,7 @@ public class HRegion implements HeapSize { // , Writable{
// do after lock
long after = EnvironmentEdgeManager.currentTimeMillis();
this.opMetrics.updateIncrementColumnValueMetrics(family, after - before);
if (flush) {
// Request a cache flush. Do it outside update lock.
requestFlush();
@ -5017,7 +5016,7 @@ public class HRegion implements HeapSize { // , Writable{
public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
ClassSize.OBJECT + // closeLock
(2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
(3 * ClassSize.ATOMIC_LONG) + // memStoreSize, numPutsWithoutWAL, dataInMemoryWithoutWAL
(3 * ClassSize.ATOMIC_LONG) + // memStoreSize, numPutsWithoutWAL, dataInMemoryWithoutWAL
ClassSize.ATOMIC_INTEGER + // lockIdGenerator
(3 * ClassSize.CONCURRENT_HASHMAP) + // lockedRows, lockIds, scannerReadPoints
WriteState.HEAP_SIZE + // writestate
@ -5371,7 +5370,7 @@ public class HRegion implements HeapSize { // , Writable{
*/
private void recordPutWithoutWal(final Map<byte [], List<KeyValue>> familyMap) {
if (numPutsWithoutWAL.getAndIncrement() == 0) {
LOG.info("writing data to region " + this +
LOG.info("writing data to region " + this +
" with WAL disabled. Data may be lost in the event of a crash.");
}

View File

@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -207,7 +206,7 @@ public class SplitTransaction {
private static IOException closedByOtherException = new IOException(
"Failed to close region: already closed by another thread");
/**
* Prepare the regions and region files.
* @param server Hosting server instance. Can be null when testing (won't try
@ -241,7 +240,7 @@ public class SplitTransaction {
this.fileSplitTimeout = testing ? this.fileSplitTimeout :
server.getConfiguration().getLong("hbase.regionserver.fileSplitTimeout",
this.fileSplitTimeout);
this.journal.add(JournalEntry.STARTED_SPLITTING);
// Set ephemeral SPLITTING znode up in zk. Mocked servers sometimes don't
// have zookeeper so don't do zk stuff if server or zookeeper is null
@ -258,7 +257,7 @@ public class SplitTransaction {
createSplitDir(this.parent.getFilesystem(), this.splitdir);
this.journal.add(JournalEntry.CREATE_SPLIT_DIR);
List<StoreFile> hstoreFilesToSplit = null;
Exception exceptionToThrow = null;
try{
@ -319,7 +318,7 @@ public class SplitTransaction {
// regions.
// We should add PONR JournalEntry before offlineParentInMeta,so even if
// OfflineParentInMeta timeout,this will cause regionserver exit,and then
// master ServerShutdownHandler will fix daughter & avoid data loss. (See
// master ServerShutdownHandler will fix daughter & avoid data loss. (See
// HBase-4562).
this.journal.add(JournalEntry.PONR);
@ -745,13 +744,13 @@ public class SplitTransaction {
while (iterator.hasPrevious()) {
JournalEntry je = iterator.previous();
switch(je) {
case STARTED_SPLITTING:
if (server != null && server.getZooKeeper() != null) {
cleanZK(server, this.parent.getRegionInfo(), false);
}
break;
case SET_SPLITTING_IN_ZK:
if (server != null && server.getZooKeeper() != null) {
cleanZK(server, this.parent.getRegionInfo(), true);
@ -859,7 +858,7 @@ public class SplitTransaction {
} catch (KeeperException.NoNodeException nn) {
if (abort) {
server.abort("Failed cleanup of " + hri.getRegionNameAsString(), nn);
}
}
} catch (KeeperException e) {
server.abort("Failed cleanup of " + hri.getRegionNameAsString(), e);
}
@ -876,8 +875,8 @@ public class SplitTransaction {
* @param region region to be created as offline
* @param serverName server event originates from
* @return Version of znode created.
* @throws KeeperException
* @throws IOException
* @throws KeeperException
* @throws IOException
*/
int createNodeSplitting(final ZooKeeperWatcher zkw, final HRegionInfo region,
final ServerName serverName) throws KeeperException, IOException {
@ -925,20 +924,20 @@ public class SplitTransaction {
* @param serverName server event originates from
* @return version of node after transition, -1 if unsuccessful transition
* @throws KeeperException if unexpected zookeeper exception
* @throws IOException
* @throws IOException
*/
private static int transitionNodeSplit(ZooKeeperWatcher zkw,
HRegionInfo parent, HRegionInfo a, HRegionInfo b, ServerName serverName,
final int znodeVersion)
throws KeeperException, IOException {
byte [] payload = Writables.getBytes(a, b);
byte [] payload = HRegionInfo.toDelimitedByteArray(a, b);
return ZKAssign.transitionNode(zkw, parent, serverName,
EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLIT,
znodeVersion, payload);
}
/**
*
*
* @param zkw zk reference
* @param parent region to be transitioned to splitting
* @param serverName server event originates from
@ -957,7 +956,7 @@ public class SplitTransaction {
HRegionInfo parent, HRegionInfo a, HRegionInfo b, ServerName serverName,
final int znodeVersion)
throws KeeperException, IOException {
byte [] payload = Writables.getBytes(a, b);
byte [] payload = HRegionInfo.toDelimitedByteArray(a, b);
return ZKAssign.transitionNode(zkw, parent, serverName,
EventType.RS_ZK_REGION_SPLIT, EventType.RS_ZK_REGION_SPLIT,
znodeVersion, payload);

View File

@ -84,7 +84,6 @@ import org.apache.hadoop.hbase.thrift.generated.TScan;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.net.DNS;
import org.apache.thrift.TException;
import org.apache.thrift.protocol.TBinaryProtocol;
@ -1375,14 +1374,12 @@ public class ThriftServerRunner implements Runnable {
}
// find region start and end keys
byte[] value = startRowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (value == null || value.length == 0) {
HRegionInfo regionInfo = HRegionInfo.getHRegionInfo(startRowResult);
if (regionInfo == null) {
throw new IOException("HRegionInfo REGIONINFO was null or " +
" empty in Meta for row="
+ Bytes.toStringBinary(row));
}
HRegionInfo regionInfo = Writables.getHRegionInfo(value);
TRegionInfo region = new TRegionInfo();
region.setStartKey(regionInfo.getStartKey());
region.setEndKey(regionInfo.getEndKey());
@ -1391,13 +1388,10 @@ public class ThriftServerRunner implements Runnable {
region.version = regionInfo.getVersion();
// find region assignment to server
value = startRowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value != null && value.length > 0) {
String hostAndPort = Bytes.toString(value);
region.setServerName(Bytes.toBytes(
Addressing.parseHostname(hostAndPort)));
region.port = Addressing.parsePort(hostAndPort);
ServerName serverName = HRegionInfo.getServerName(startRowResult);
if (serverName != null) {
region.setServerName(Bytes.toBytes(serverName.getHostname()));
region.port = serverName.getPort();
}
return region;
} catch (IOException e) {

View File

@ -62,7 +62,7 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@ -724,7 +724,7 @@ public class HBaseFsck {
/**
* This borrows code from MasterFileSystem.bootstrap()
*
*
* @return an open .META. HRegion
*/
private HRegion createNewRootAndMeta() throws IOException {
@ -748,9 +748,9 @@ public class HBaseFsck {
}
/**
* Generate set of puts to add to new meta. This expects the tables to be
* Generate set of puts to add to new meta. This expects the tables to be
* clean with no overlaps or holes. If there are any problems it returns null.
*
*
* @return An array list of puts to do in bulk, null if tables have problems
*/
private ArrayList<Put> generatePuts(SortedMap<String, TableInfo> tablesInfo) throws IOException {
@ -781,9 +781,7 @@ public class HBaseFsck {
// add the row directly to meta.
HbckInfo hi = his.iterator().next();
HRegionInfo hri = hi.getHdfsHRI(); // hi.metaEntry;
Put p = new Put(hri.getRegionName());
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
Put p = MetaEditor.makePutFromRegionInfo(hri);
puts.add(p);
}
}
@ -803,7 +801,7 @@ public class HBaseFsck {
/**
* Rebuilds meta from information in hdfs/fs. Depends on configuration
* settings passed into hbck constructor to point to a particular fs/dir.
*
*
* @param fix flag that determines if method should attempt to fix holes
* @return true if successful, false if attempt failed.
*/
@ -989,7 +987,7 @@ public class HBaseFsck {
Path backupTableDir= new Path(backupHbaseDir, tableName);
boolean success = fs.rename(tableDir, backupTableDir);
if (!success) {
throw new IOException("Failed to move " + tableName + " from "
throw new IOException("Failed to move " + tableName + " from "
+ tableDir.getName() + " to " + backupTableDir.getName());
}
} else {
@ -1020,7 +1018,7 @@ public class HBaseFsck {
} catch (IOException ioe) {
LOG.fatal("... failed to sideline root and meta and failed to restore "
+ "prevoius state. Currently in inconsistent state. To restore "
+ "try to rename -ROOT- in " + backupDir.getName() + " to "
+ "try to rename -ROOT- in " + backupDir.getName() + " to "
+ hbaseDir.getName() + ".", ioe);
}
throw e; // throw original exception
@ -1194,7 +1192,7 @@ public class HBaseFsck {
for (ServerName rsinfo: regionServerList) {
workItems.add(new WorkItemRegion(this, rsinfo, errors, connection));
}
workFutures = executor.invokeAll(workItems);
for(int i=0; i<workFutures.size(); i++) {
@ -1260,12 +1258,10 @@ public class HBaseFsck {
d.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER);
mutations.add(d);
Put p = new Put(hi.metaEntry.getRegionName());
HRegionInfo hri = new HRegionInfo(hi.metaEntry);
hri.setOffline(false);
hri.setSplit(false);
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
Put p = MetaEditor.makePutFromRegionInfo(hri);
mutations.add(p);
meta.mutateRow(mutations);
@ -1335,7 +1331,6 @@ public class HBaseFsck {
* the offline ipc call exposed on the master (<0.90.5, <0.92.0) a master
* restart or failover may be required.
*/
@SuppressWarnings("deprecation")
private void closeRegion(HbckInfo hi) throws IOException, InterruptedException {
if (hi.metaEntry == null && hi.hdfsEntry == null) {
undeployRegions(hi);
@ -1348,19 +1343,15 @@ public class HBaseFsck {
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
Result r = meta.get(get);
byte[] value = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
byte[] startcodeBytes = r.getValue(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
if (value == null || startcodeBytes == null) {
ServerName serverName = HRegionInfo.getServerName(r);
if (serverName == null) {
errors.reportError("Unable to close region "
+ hi.getRegionNameAsString() + " because meta does not "
+ "have handle to reach it.");
return;
}
long startcode = Bytes.toLong(startcodeBytes);
ServerName hsa = new ServerName(Bytes.toString(value), startcode);
byte[] hriVal = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
HRegionInfo hri= Writables.getHRegionInfoOrNull(hriVal);
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri == null) {
LOG.warn("Unable to close region " + hi.getRegionNameAsString()
+ " because META had invalid or missing "
@ -1371,7 +1362,7 @@ public class HBaseFsck {
}
// close the region -- close files and remove assignment
HBaseFsckRepair.closeRegionSilentlyAndWait(admin, hsa, hri);
HBaseFsckRepair.closeRegionSilentlyAndWait(admin, serverName, hri);
}
private void tryAssignmentRepair(HbckInfo hbi, String msg) throws IOException,
@ -1723,7 +1714,7 @@ public class HBaseFsck {
return;
}
// if not the absolute end key, check for cycle
// if not the absolute end key, check for cycle
if (Bytes.compareTo(hir.getStartKey(), hir.getEndKey()) > 0) {
errors.reportError(
ERROR_CODE.REGION_CYCLE,
@ -1773,7 +1764,7 @@ public class HBaseFsck {
"Last region should end with an empty key. You need to "
+ "create a new region and regioninfo in HDFS to plug the hole.", getTableInfo());
}
@Override
public void handleDegenerateRegion(HbckInfo hi) throws IOException{
errors.reportError(ERROR_CODE.DEGENERATE_REGION,
@ -1872,7 +1863,7 @@ public class HBaseFsck {
+ " " + region);
fixes++;
}
/**
* There is a hole in the hdfs regions that violates the table integrity
* rules. Create a new empty region that patches the hole.
@ -2131,7 +2122,7 @@ public class HBaseFsck {
if (prevKey != null) {
handler.handleRegionEndKeyNotEmpty(prevKey);
}
for (Collection<HbckInfo> overlap : overlapGroups.asMap().values()) {
handler.handleOverlapGroup(overlap);
}
@ -2159,7 +2150,7 @@ public class HBaseFsck {
/**
* This dumps data in a visually reasonable way for visual debugging
*
*
* @param splits
* @param regions
*/
@ -2348,7 +2339,7 @@ public class HBaseFsck {
// record the latest modification of this META record
long ts = Collections.max(result.list(), comp).getTimestamp();
Pair<HRegionInfo, ServerName> pair = MetaReader.parseCatalogResult(result);
Pair<HRegionInfo, ServerName> pair = HRegionInfo.getHRegionInfoAndServerName(result);
if (pair == null || pair.getFirst() == null) {
emptyRegionInfoQualifiers.add(result);
return true;
@ -2695,7 +2686,7 @@ public class HBaseFsck {
errorTables.add(table);
reportError(errorCode, message);
}
public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table,
HbckInfo info) {
errorTables.add(table);
@ -2821,7 +2812,7 @@ public class HBaseFsck {
HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName());
hbi.addServer(r, rsinfo);
}
} catch (IOException e) { // unable to connect to the region server.
} catch (IOException e) { // unable to connect to the region server.
errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "RegionServer: " + rsinfo.getServerName() +
" Unable to fetch region information. " + e);
throw e;
@ -2851,7 +2842,7 @@ public class HBaseFsck {
private ErrorReporter errors;
private FileSystem fs;
WorkItemHdfsDir(HBaseFsck hbck, FileSystem fs, ErrorReporter errors,
WorkItemHdfsDir(HBaseFsck hbck, FileSystem fs, ErrorReporter errors,
FileStatus status) {
this.hbck = hbck;
this.fs = fs;
@ -3049,7 +3040,7 @@ public class HBaseFsck {
public boolean shouldFixVersionFile() {
return fixVersionFile;
}
public void setSidelineBigOverlaps(boolean sbo) {
this.sidelineBigOverlaps = sbo;
}
@ -3119,7 +3110,7 @@ public class HBaseFsck {
}
/**
*
*
* @param sidelineDir - HDFS path to sideline data
*/
public void setSidelineDir(String sidelineDir) {

View File

@ -34,11 +34,11 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -174,11 +174,8 @@ public class HBaseFsckRepair {
*/
public static void fixMetaHoleOnline(Configuration conf,
HRegionInfo hri) throws IOException {
Put p = new Put(hri.getRegionName());
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
meta.put(p);
MetaEditor.addRegionToMeta(meta, hri);
meta.close();
}

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
@ -257,14 +258,12 @@ class HMerge {
if (results == null) {
return null;
}
byte[] regionInfoValue = results.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (regionInfoValue == null || regionInfoValue.length == 0) {
HRegionInfo region = HRegionInfo.getHRegionInfo(results);
if (region == null) {
throw new NoSuchElementException("meta region entry missing " +
Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
Bytes.toString(HConstants.REGIONINFO_QUALIFIER));
}
HRegionInfo region = Writables.getHRegionInfo(regionInfoValue);
if (!Bytes.equals(region.getTableName(), this.tableName)) {
return null;
}
@ -333,10 +332,7 @@ class HMerge {
}
newRegion.getRegionInfo().setOffline(true);
Put put = new Put(newRegion.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(newRegion.getRegionInfo()));
table.put(put);
MetaEditor.addRegionToMeta(table, newRegion.getRegionInfo());
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: "
@ -376,7 +372,7 @@ class HMerge {
do {
hasMore = rootScanner.next(results);
for(KeyValue kv: results) {
HRegionInfo info = Writables.getHRegionInfoOrNull(kv.getValue());
HRegionInfo info = HRegionInfo.parseFromOrNull(kv.getValue());
if (info != null) {
metaRegions.add(info);
}
@ -428,9 +424,8 @@ class HMerge {
}
HRegionInfo newInfo = newRegion.getRegionInfo();
newInfo.setOffline(true);
Put put = new Put(newRegion.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(newInfo));
Put put = MetaEditor.makePutFromRegionInfo(newInfo);
root.put(put);
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + Bytes.toStringBinary(newRegion.getRegionName()));

View File

@ -20,6 +20,8 @@
package org.apache.hadoop.hbase.util;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -32,12 +34,12 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
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.MasterNotRunningException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.io.WritableComparator;
@ -47,9 +49,6 @@ import org.apache.hadoop.util.ToolRunner;
import com.google.common.base.Preconditions;
import java.io.IOException;
import java.util.List;
/**
* Utility that can merge any two regions in the same table: adjacent,
* overlapping or disjoint.
@ -153,15 +152,15 @@ public class Merge extends Configured implements Tool {
HRegion rootRegion = utils.getRootRegion();
Get get = new Get(region1);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
List<KeyValue> cells1 = rootRegion.get(get, null).list();
Preconditions.checkState(cells1 != null, "First region cells can not be null");
HRegionInfo info1 = Writables.getHRegionInfo(cells1.get(0).getValue());
Result result1 = rootRegion.get(get, null);
Preconditions.checkState(!result1.isEmpty(), "First region cells can not be null");
HRegionInfo info1 = HRegionInfo.getHRegionInfo(result1);
get = new Get(region2);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
List<KeyValue> cells2 = rootRegion.get(get, null).list();
Preconditions.checkState(cells2 != null, "Second region cells can not be null");
HRegionInfo info2 = Writables.getHRegionInfo(cells2.get(0).getValue());
Result result2 = rootRegion.get(get, null);
Preconditions.checkState(!result2.isEmpty(), "Second region cells can not be null");
HRegionInfo info2 = HRegionInfo.getHRegionInfo(result2);
HRegion merged = merge(HTableDescriptor.META_TABLEDESC, info1, rootRegion, info2, rootRegion);
LOG.info("Adding " + merged.getRegionInfo() + " to " +
rootRegion.getRegionInfo());
@ -224,10 +223,10 @@ public class Merge extends Configured implements Tool {
HRegion metaRegion1 = this.utils.getMetaRegion(meta1);
Get get = new Get(region1);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
List<KeyValue> cells1 = metaRegion1.get(get, null).list();
Preconditions.checkState(cells1 != null,
Result result1 = metaRegion1.get(get, null);
Preconditions.checkState(!result1.isEmpty(),
"First region cells can not be null");
HRegionInfo info1 = Writables.getHRegionInfo(cells1.get(0).getValue());
HRegionInfo info1 = HRegionInfo.getHRegionInfo(result1);
if (info1 == null) {
throw new NullPointerException("info1 is null using key " +
Bytes.toStringBinary(region1) + " in " + meta1);
@ -241,10 +240,10 @@ public class Merge extends Configured implements Tool {
}
get = new Get(region2);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
List<KeyValue> cells2 = metaRegion2.get(get, null).list();
Preconditions.checkState(cells2 != null,
Result result2 = metaRegion2.get(get, null);
Preconditions.checkState(!result2.isEmpty(),
"Second region cells can not be null");
HRegionInfo info2 = Writables.getHRegionInfo(cells2.get(0).getValue());
HRegionInfo info2 = HRegionInfo.getHRegionInfo(result2);
if (info2 == null) {
throw new NullPointerException("info2 is null using key " + meta2);
}

View File

@ -38,6 +38,7 @@ 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.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
@ -220,7 +221,7 @@ public class MetaUtils {
hasNext = s.next(results);
HRegionInfo info = null;
for (KeyValue kv: results) {
info = Writables.getHRegionInfoOrNull(kv.getValue());
info = HRegionInfo.parseFromOrNull(kv.getValue());
if (info == null) {
LOG.warn("Region info is null for row " +
Bytes.toStringBinary(kv.getRow()) + " in table " +
@ -302,16 +303,13 @@ public class MetaUtils {
if(kvs.length <= 0) {
throw new IOException("no information for row " + Bytes.toString(row));
}
byte [] value = kvs[0].getValue();
if (value == null) {
HRegionInfo info = HRegionInfo.getHRegionInfo(res);
if (info == null) {
throw new IOException("no information for row " + Bytes.toString(row));
}
HRegionInfo info = Writables.getHRegionInfo(value);
Put put = new Put(row);
info.setOffline(onlineOffline);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(info));
t.put(put);
MetaEditor.addRegionToMeta(t, info);
Delete delete = new Delete(row);
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
@ -338,20 +336,17 @@ public class MetaUtils {
if(kvs.length <= 0) {
return;
}
byte [] value = kvs[0].getValue();
if (value == null) {
HRegionInfo h = HRegionInfo.getHRegionInfo(res);
if (h == null) {
return;
}
HRegionInfo h = Writables.getHRegionInfoOrNull(value);
LOG.debug("Old " + Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " for " +
hri.toString() + " in " + r.toString() + " is: " + h.toString());
}
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
Put put = MetaEditor.makePutFromRegionInfo(hri);
r.put(put);
if (LOG.isDebugEnabled()) {
@ -362,12 +357,11 @@ public class MetaUtils {
if(kvs.length <= 0) {
return;
}
byte [] value = kvs[0].getValue();
if (value == null) {
HRegionInfo h = HRegionInfo.getHRegionInfo(res);
if (h == null) {
return;
}
HRegionInfo h = Writables.getHRegionInfoOrNull(value);
LOG.debug("New " + Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
LOG.debug("New " + Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " for " +
hri.toString() + " in " + r.toString() + " is: " + h.toString());
}

View File

@ -19,11 +19,6 @@
*/
package org.apache.hadoop.hbase.util;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Writable;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
@ -32,6 +27,10 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Writable;
/**
* Utility class with methods for manipulating Writable objects
*/
@ -138,58 +137,6 @@ public class Writables {
}
}
/**
* @param bytes serialized bytes
* @return A HRegionInfo instance built out of passed <code>bytes</code>.
* @throws IOException e
* @deprecated Use {@link HRegionInfo#parseFrom(byte[])}
*/
public static HRegionInfo getHRegionInfo(final byte [] bytes)
throws IOException {
return (HRegionInfo)getWritable(bytes, new HRegionInfo());
}
/**
* @param bytes serialized bytes
* @return All the hregioninfos that are in the byte array. Keeps reading
* till we hit the end.
* @throws IOException e
*/
public static List<HRegionInfo> getHRegionInfos(final byte [] bytes,
final int offset, final int length)
throws IOException {
if (bytes == null) {
throw new IllegalArgumentException("Can't build a writable with empty " +
"bytes array");
}
DataInputBuffer in = new DataInputBuffer();
List<HRegionInfo> hris = new ArrayList<HRegionInfo>();
try {
in.reset(bytes, offset, length);
while (in.available() > 0) {
HRegionInfo hri = new HRegionInfo();
hri.readFields(in);
hris.add(hri);
}
} finally {
in.close();
}
return hris;
}
/**
* @param bytes serialized bytes
* @return A HRegionInfo instance built out of passed <code>bytes</code>
* or <code>null</code> if passed bytes are null or an empty array.
* @throws IOException e
* @deprecated Use {@link HRegionInfo#parseFromOrNull(byte[])}
*/
public static HRegionInfo getHRegionInfoOrNull(final byte [] bytes)
throws IOException {
return (bytes == null || bytes.length <= 0)?
null : getHRegionInfo(bytes);
}
/**
* Copy one Writable to another. Copies bytes using data streams.
* @param src Source Writable

View File

@ -459,7 +459,7 @@ EOF
def to_string(column, kv, maxlength = -1)
if is_meta_table?
if column == 'info:regioninfo' or column == 'info:splitA' or column == 'info:splitB'
hri = org.apache.hadoop.hbase.util.Writables.getHRegionInfoOrNull(kv.getValue)
hri = org.apache.hadoop.hbase.HRegionInfo.parseFromOrNull(kv.getValue)
return "timestamp=%d, value=%s" % [kv.getTimestamp, hri.toString]
end
if column == 'info:serverstartcode'

View File

@ -53,6 +53,7 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@ -81,7 +82,6 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@ -1188,11 +1188,7 @@ public class HBaseTestingUtility {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(table.getTableName(),
startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
LOG.info("createMultiRegions: inserted " + hri.toString());
MetaEditor.addRegionToMeta(meta, hri);
newRegions.add(hri);
count++;
}
@ -1238,11 +1234,7 @@ public class HBaseTestingUtility {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(htd.getName(), startKeys[i],
startKeys[j]);
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
LOG.info("createMultiRegionsInMeta: inserted " + hri.toString());
MetaEditor.addRegionToMeta(meta, hri);
newRegions.add(hri);
}
@ -1281,13 +1273,13 @@ public class HBaseTestingUtility {
List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan());
for (Result result : s) {
byte[] val = result.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (val == null) {
HRegionInfo info = HRegionInfo.getHRegionInfo(result);
if (info == null) {
LOG.error("No region info for row " + Bytes.toString(result.getRow()));
// TODO figure out what to do for this new hosed case.
continue;
}
HRegionInfo info = Writables.getHRegionInfo(val);
if (Bytes.compareTo(info.getTableName(), tableName) == 0) {
LOG.info("getMetaTableRows: row -> " +
Bytes.toStringBinary(result.getRow()) + info);
@ -1390,7 +1382,7 @@ public class HBaseTestingUtility {
// Needed for TestImportTsv.
conf.set("mapred.job.tracker", jobConf.get("mapred.job.tracker"));
// this for mrv2 support; mr1 ignores this
// this for mrv2 support; mr1 ignores this
conf.set("mapreduce.framework.name", "yarn");
String rmAdress = jobConf.get("yarn.resourcemanager.address");
if (rmAdress != null) {

View File

@ -20,20 +20,18 @@
package org.apache.hadoop.hbase;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.hbase.HServerLoad092.RegionLoad;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
@ -41,10 +39,10 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowLock;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.RowFilter;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -115,22 +113,30 @@ public class TestSerialization {
*/
@Test public void testRegionInfo() throws Exception {
HRegionInfo hri = createRandomRegion("testRegionInfo");
byte [] hrib = Writables.getBytes(hri);
HRegionInfo deserializedHri =
(HRegionInfo)Writables.getWritable(hrib, new HRegionInfo());
//test toByteArray()
byte [] hrib = hri.toByteArray();
HRegionInfo deserializedHri = HRegionInfo.parseFrom(hrib);
assertEquals(hri.getEncodedName(), deserializedHri.getEncodedName());
//assertEquals(hri.getTableDesc().getFamilies().size(),
// deserializedHri.getTableDesc().getFamilies().size());
assertEquals(hri, deserializedHri);
//test toDelimitedByteArray()
hrib = hri.toDelimitedByteArray();
DataInputBuffer buf = new DataInputBuffer();
try {
buf.reset(hrib, hrib.length);
deserializedHri = HRegionInfo.parseFrom(buf);
assertEquals(hri.getEncodedName(), deserializedHri.getEncodedName());
assertEquals(hri, deserializedHri);
} finally {
buf.close();
}
}
@Test public void testRegionInfos() throws Exception {
HRegionInfo hri = createRandomRegion("testRegionInfos");
byte [] hrib = Writables.getBytes(hri);
byte [] triple = new byte [3 * hrib.length];
System.arraycopy(hrib, 0, triple, 0, hrib.length);
System.arraycopy(hrib, 0, triple, hrib.length, hrib.length);
System.arraycopy(hrib, 0, triple, hrib.length * 2, hrib.length);
List<HRegionInfo> regions = Writables.getHRegionInfos(triple, 0, triple.length);
byte[] triple = HRegionInfo.toDelimitedByteArray(hri, hri, hri);
List<HRegionInfo> regions = HRegionInfo.parseDelimitedFrom(triple, 0, triple.length);
assertTrue(regions.size() == 3);
assertTrue(regions.get(0).equals(regions.get(1)));
assertTrue(regions.get(0).equals(regions.get(2)));

View File

@ -0,0 +1,125 @@
/**
* 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.catalog;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Mocking utility for common META functionality
*/
public class MetaMockingUtil {
/**
* Returns a Result object constructed from the given region information simulating
* a catalog table result.
* @param region the HRegionInfo object or null
* @return A mocked up Result that fakes a Get on a row in the <code>.META.</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResult(final HRegionInfo region)
throws IOException {
return getMetaTableRowResult(region, null, null, null);
}
/**
* Returns a Result object constructed from the given region information simulating
* a catalog table result.
* @param region the HRegionInfo object or null
* @param ServerName to use making startcode and server hostname:port in meta or null
* @return A mocked up Result that fakes a Get on a row in the <code>.META.</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResult(final HRegionInfo region, final ServerName sn)
throws IOException {
return getMetaTableRowResult(region, sn, null, null);
}
/**
* Returns a Result object constructed from the given region information simulating
* a catalog table result.
* @param region the HRegionInfo object or null
* @param ServerName to use making startcode and server hostname:port in meta or null
* @param splita daughter region or null
* @param splitb daughter region or null
* @return A mocked up Result that fakes a Get on a row in the <code>.META.</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResult(HRegionInfo region, final ServerName sn,
HRegionInfo splita, HRegionInfo splitb) throws IOException {
List<KeyValue> kvs = new ArrayList<KeyValue>();
if (region != null) {
kvs.add(new KeyValue(
region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
region.toByteArray()));
}
if (sn != null) {
kvs.add(new KeyValue(region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(sn.getHostAndPort())));
kvs.add(new KeyValue(region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(sn.getStartcode())));
}
if (splita != null) {
kvs.add(new KeyValue(
region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
splita.toByteArray()));
}
if (splitb != null) {
kvs.add(new KeyValue(
region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
splitb.toByteArray()));
}
//important: sort the kvs so that binary search work
Collections.sort(kvs, KeyValue.META_COMPARATOR);
return new Result(kvs);
}
/**
* @param sn ServerName to use making startcode and server in meta
* @param hri Region to serialize into HRegionInfo
* @return A mocked up Result that fakes a Get on a row in the <code>.META.</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResultAsSplitRegion(final HRegionInfo hri, final ServerName sn)
throws IOException {
hri.setOffline(true);
hri.setSplit(true);
return getMetaTableRowResult(hri, sn);
}
}

View File

@ -23,8 +23,6 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.net.ConnectException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -38,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
@ -50,19 +47,18 @@ import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.client.ServerCallable;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
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.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.Progressable;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -103,7 +99,7 @@ public class TestCatalogTracker {
public void abort(String why, Throwable e) {
LOG.info(why, e);
}
@Override
public boolean isAborted() {
return false;
@ -127,9 +123,9 @@ public class TestCatalogTracker {
/**
* Test that we get notification if .META. moves.
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
@Test public void testThatIfMETAMovesWeAreNotified()
throws IOException, InterruptedException, KeeperException {
@ -410,7 +406,7 @@ public class TestCatalogTracker {
/**
* Test waiting on meta w/ no timeout specified.
* @throws Exception
* @throws Exception
*/
@Ignore // Can't make it work reliably on all platforms; mockito gets confused
// Throwing: org.mockito.exceptions.misusing.WrongTypeOfReturnValue:
@ -517,20 +513,10 @@ public class TestCatalogTracker {
/**
* @return A mocked up Result that fakes a Get on a row in the
* <code>.META.</code> table.
* @throws IOException
* @throws IOException
*/
private Result getMetaTableRowResult() throws IOException {
List<KeyValue> kvs = new ArrayList<KeyValue>();
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(HRegionInfo.FIRST_META_REGIONINFO)));
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(SN.getHostAndPort())));
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(SN.getStartcode())));
return new Result(kvs);
return MetaMockingUtil.getMetaTableRowResult(HRegionInfo.FIRST_META_REGIONINFO, SN);
}
private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {

View File

@ -1,6 +1,4 @@
/**
* Copyright 2010 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@ -17,19 +15,19 @@
* 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.*;
package org.apache.hadoop.hbase.catalog;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import junit.framework.Assert;
import junit.framework.AssertionFailedError;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -38,37 +36,65 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
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.MediumTests;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.DataOutputBuffer;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test migration that removes HTableDescriptor from HRegionInfo moving the
* meta version from no version to {@link MetaReader#META_VERSION}.
* Test migration that changes HRI serialization into PB. Tests by bringing up a cluster from actual
* data from a 0.92 cluster, as well as manually downgrading and then upgrading the META info.
* @deprecated Remove after 0.96
*/
@Category(MediumTests.class)
public class TestMetaMigrationRemovingHTD {
static final Log LOG = LogFactory.getLog(TestMetaMigrationRemovingHTD.class);
@Deprecated
public class TestMetaMigrationConvertingToPB {
static final Log LOG = LogFactory.getLog(TestMetaMigrationConvertingToPB.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final static String TESTTABLE = "TestTable";
private final static int ROWCOUNT = 100;
private final static int ROW_COUNT = 100;
private final static int REGION_COUNT = 9; //initial number of regions of the TestTable
private static final int META_VERSION_092 = 0;
/*
* This test uses a tgz file named "TestMetaMigrationConvertingToPB.tgz" under
* hbase-server/src/test/data which contains file data from a 0.92 cluster.
* The cluster has a table named "TestTable", which has 100 rows. 0.94 has same
* META structure, so it should be the same.
*
* hbase(main):001:0> create 'TestTable', 'f1'
* hbase(main):002:0> for i in 1..100
* hbase(main):003:1> put 'TestTable', "row#{i}", "f1:c1", i
* hbase(main):004:1> end
*
* There are 9 regions in the table
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Start up our mini cluster on top of an 0.90 root.dir that has data from
// a 0.90 hbase run -- it has a table with 100 rows in it -- and see if
// we can migrate from 0.90.
// Start up our mini cluster on top of an 0.92 root.dir that has data from
// a 0.92 hbase run -- it has a table with 100 rows in it -- and see if
// we can migrate from 0.92
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.startMiniDFSCluster(1);
Path testdir = TEST_UTIL.getDataTestDir("TestMetaMigrationRemovingHTD");
Path testdir = TEST_UTIL.getDataTestDir("TestMetaMigrationConvertToPB");
// Untar our test dir.
File untar = untar(new File(testdir.toString()));
// Now copy the untar up into hdfs so when we start hbase, we'll run from it.
@ -98,14 +124,14 @@ public class TestMetaMigrationRemovingHTD {
}
// Assert that we find all 100 rows that are in the data we loaded. If
// so then we must have migrated it from 0.90 to 0.92.
Assert.assertEquals(ROWCOUNT, count);
Assert.assertEquals(ROW_COUNT, count);
scanner.close();
t.close();
}
private static File untar(final File testdir) throws IOException {
// Find the src data under src/test/data
final String datafile = "hbase-4388-root.dir";
final String datafile = "TestMetaMigrationConvertToPB";
String srcTarFile =
System.getProperty("project.build.testSourceDirectory", "src/test") +
File.separator + "data" + File.separator + datafile + ".tgz";
@ -139,52 +165,55 @@ public class TestMetaMigrationRemovingHTD {
@Test
public void testMetaUpdatedFlagInROOT() throws Exception {
boolean metaUpdated = MetaMigrationRemovingHTD.
isMetaHRIUpdated(TEST_UTIL.getMiniHBaseCluster().getMaster());
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
boolean metaUpdated = MetaMigrationConvertingToPB.
isMetaHRIUpdated(master.getCatalogTracker());
assertEquals(true, metaUpdated);
verifyMetaRowsAreUpdated(master.getCatalogTracker());
}
@Test
public void testMetaMigration() throws Exception {
LOG.info("Starting testMetaWithLegacyHRI");
LOG.info("Starting testMetaMigration");
final byte [] FAMILY = Bytes.toBytes("family");
HTableDescriptor htd = new HTableDescriptor("testMetaMigration");
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
htd.addFamily(hcd);
Configuration conf = TEST_UTIL.getConfiguration();
createMultiRegionsWithLegacyHRI(conf, htd, FAMILY,
new byte[][]{
HConstants.EMPTY_START_ROW,
Bytes.toBytes("region_a"),
Bytes.toBytes("region_b")});
byte[][] regionNames = new byte[][]{
HConstants.EMPTY_START_ROW,
Bytes.toBytes("region_a"),
Bytes.toBytes("region_b")};
createMultiRegionsWithWritableSerialization(conf, htd.getName(), regionNames);
CatalogTracker ct =
TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker();
// Erase the current version of root meta for this test.
undoVersionInMeta();
undoVersionInRoot(ct);
MetaReader.fullScanMetaAndPrint(ct);
LOG.info("Meta Print completed.testUpdatesOnMetaWithLegacyHRI");
LOG.info("Meta Print completed.testMetaMigration");
Set<HTableDescriptor> htds =
MetaMigrationRemovingHTD.updateMetaWithNewRegionInfo(
long numMigratedRows = MetaMigrationConvertingToPB.updateMeta(
TEST_UTIL.getHBaseCluster().getMaster());
MetaReader.fullScanMetaAndPrint(ct);
// Should be one entry only and it should be for the table we just added.
assertEquals(1, htds.size());
assertTrue(htds.contains(htd));
assertEquals(regionNames.length, numMigratedRows);
// Assert that the flag in ROOT is updated to reflect the correct status
boolean metaUpdated =
MetaMigrationRemovingHTD.isMetaHRIUpdated(
TEST_UTIL.getMiniHBaseCluster().getMaster());
MetaMigrationConvertingToPB.isMetaHRIUpdated(
TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker());
assertEquals(true, metaUpdated);
verifyMetaRowsAreUpdated(ct);
}
/**
* This test assumes a master crash/failure during the meta migration process
* and attempts to continue the meta migration process when a new master takes over.
* When a master dies during the meta migration we will have some rows of
* META.CatalogFamily updated with new HRI, (i.e HRI with out HTD) and some
* still hanging with legacy HRI. (i.e HRI with HTD). When the backup master/ or
* fresh start of master attempts the migration it will encouter some rows of META
* META.CatalogFamily updated with PB serialization and some
* still hanging with writable serialization. When the backup master/ or
* fresh start of master attempts the migration it will encounter some rows of META
* already updated with new HRI and some still legacy. This test will simulate this
* scenario and validates that the migration process can safely skip the updated
* rows and migrate any pending rows at startup.
@ -198,72 +227,76 @@ public class TestMetaMigrationRemovingHTD {
htd.addFamily(hcd);
Configuration conf = TEST_UTIL.getConfiguration();
// Create 10 New regions.
createMultiRegionsWithNewHRI(conf, htd, FAMILY, 10);
createMultiRegionsWithPBSerialization(conf, htd.getName(), 10);
// Create 10 Legacy regions.
createMultiRegionsWithLegacyHRI(conf, htd, FAMILY, 10);
createMultiRegionsWithWritableSerialization(conf, htd.getName(), 10);
CatalogTracker ct =
TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker();
// Erase the current version of root meta for this test.
undoVersionInMeta();
MetaMigrationRemovingHTD.updateRootWithMetaMigrationStatus(ct);
//MetaReader.fullScanMetaAndPrint(ct);
undoVersionInRoot(ct);
MetaReader.fullScanMetaAndPrint(ct);
LOG.info("Meta Print completed.testUpdatesOnMetaWithLegacyHRI");
Set<HTableDescriptor> htds =
MetaMigrationRemovingHTD.updateMetaWithNewRegionInfo(
TEST_UTIL.getHBaseCluster().getMaster());
assertEquals(1, htds.size());
assertTrue(htds.contains(htd));
long numMigratedRows =
MetaMigrationConvertingToPB.updateRootAndMetaIfNecessary(
TEST_UTIL.getHBaseCluster().getMaster());
assertEquals(numMigratedRows, 10);
// Assert that the flag in ROOT is updated to reflect the correct status
boolean metaUpdated = MetaMigrationRemovingHTD.
isMetaHRIUpdated(TEST_UTIL.getMiniHBaseCluster().getMaster());
boolean metaUpdated = MetaMigrationConvertingToPB.
isMetaHRIUpdated(TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker());
assertEquals(true, metaUpdated);
LOG.info("END testMetaWithLegacyHRI");
}
private void undoVersionInMeta() throws IOException {
Delete d = new Delete(HRegionInfo.ROOT_REGIONINFO.getRegionName());
// Erase the current version of root meta for this test.
d.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.META_VERSION_QUALIFIER);
HTable rootTable =
new HTable(TEST_UTIL.getConfiguration(), HConstants.ROOT_TABLE_NAME);
try {
rootTable.delete(d);
} finally {
rootTable.close();
}
}
verifyMetaRowsAreUpdated(ct);
public static void assertEquals(int expected, int actual) {
if (expected != actual) {
throw new AssertionFailedError("expected:<" +
expected + "> but was:<" +
actual + ">");
}
LOG.info("END testMasterCrashDuringMetaMigration");
}
public static void assertEquals(boolean expected, boolean actual) {
if (expected != actual) {
throw new AssertionFailedError("expected:<" +
expected + "> but was:<" +
actual + ">");
}
}
/**
* @param c
* @param htd
* @param family
* @param numRegions
* @return
* @throws IOException
* @deprecated Just for testing migration of meta from 0.90 to 0.92... will be
* removed thereafter
* Verify that every META row is updated
*/
public int createMultiRegionsWithLegacyHRI(final Configuration c,
final HTableDescriptor htd, final byte [] family, int numRegions)
throws IOException {
void verifyMetaRowsAreUpdated(CatalogTracker catalogTracker)
throws IOException {
List<Result> results = MetaReader.fullScan(catalogTracker);
assertTrue(results.size() >= REGION_COUNT);
for (Result result : results) {
byte[] hriBytes = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
assertTrue(hriBytes != null && hriBytes.length > 0);
assertTrue(MetaMigrationConvertingToPB.isMigrated(hriBytes));
byte[] splitA = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER);
if (splitA != null && splitA.length > 0) {
assertTrue(MetaMigrationConvertingToPB.isMigrated(splitA));
}
byte[] splitB = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER);
if (splitB != null && splitB.length > 0) {
assertTrue(MetaMigrationConvertingToPB.isMigrated(splitB));
}
}
}
/** Changes the version of META to 0 to simulate 0.92 and 0.94 clusters*/
private void undoVersionInRoot(CatalogTracker ct) throws IOException {
Put p = new Put(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
p.add(HConstants.CATALOG_FAMILY, HConstants.META_VERSION_QUALIFIER,
Bytes.toBytes(META_VERSION_092));
MetaEditor.putToRootTable(ct, p);
LOG.info("Downgraded -ROOT- meta version=" + META_VERSION_092);
}
/**
* Inserts multiple regions into META using Writable serialization instead of PB
*/
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final byte[] tableName, int numRegions) throws IOException {
if (numRegions < 3) throw new IOException("Must create at least 3 regions");
byte [] startKey = Bytes.toBytes("aaaaa");
byte [] endKey = Bytes.toBytes("zzzzz");
@ -273,41 +306,38 @@ public class TestMetaMigrationRemovingHTD {
regionStartKeys[i+1] = splitKeys[i];
}
regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
return createMultiRegionsWithLegacyHRI(c, htd, family, regionStartKeys);
return createMultiRegionsWithWritableSerialization(c, tableName, regionStartKeys);
}
/**
* @param c
* @param htd
* @param columnFamily
* @param startKeys
* @return
* @throws IOException
* @deprecated Just for testing migration of meta from 0.90 to 0.92... will be
* removed thereafter
* Inserts multiple regions into META using Writable serialization instead of PB
*/
public int createMultiRegionsWithLegacyHRI(final Configuration c,
final HTableDescriptor htd, final byte[] columnFamily, byte [][] startKeys)
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final byte[] tableName, byte [][] startKeys)
throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
if(!htd.hasFamily(columnFamily)) {
HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
htd.addFamily(hcd);
}
List<HRegionInfo090x> newRegions
= new ArrayList<HRegionInfo090x>(startKeys.length);
List<HRegionInfo> newRegions
= new ArrayList<HRegionInfo>(startKeys.length);
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo090x hri = new HRegionInfo090x(htd,
startKeys[i], startKeys[j]);
HRegionInfo hri = new HRegionInfo(tableName, startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
put.setWriteToWAL(false);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
getBytes(hri)); //this is the old Writable serialization
//also add the region as it's daughters
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
getBytes(hri)); //this is the old Writable serialization
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
getBytes(hri)); //this is the old Writable serialization
meta.put(put);
LOG.info("createMultiRegions: PUT inserted " + hri.toString());
LOG.info("createMultiRegionsWithWritableSerialization: PUT inserted " + hri.toString());
newRegions.add(hri);
count++;
@ -316,8 +346,24 @@ public class TestMetaMigrationRemovingHTD {
return count;
}
int createMultiRegionsWithNewHRI(final Configuration c,
final HTableDescriptor htd, final byte [] family, int numRegions)
@Deprecated
private byte[] getBytes(HRegionInfo hri) throws IOException {
DataOutputBuffer out = new DataOutputBuffer();
try {
hri.write(out);
return out.getData();
} finally {
if (out != null) {
out.close();
}
}
}
/**
* Inserts multiple regions into META using PB serialization
*/
int createMultiRegionsWithPBSerialization(final Configuration c,
final byte[] tableName, int numRegions)
throws IOException {
if (numRegions < 3) throw new IOException("Must create at least 3 regions");
byte [] startKey = Bytes.toBytes("aaaaa");
@ -328,31 +374,27 @@ public class TestMetaMigrationRemovingHTD {
regionStartKeys[i+1] = splitKeys[i];
}
regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
return createMultiRegionsWithNewHRI(c, htd, family, regionStartKeys);
return createMultiRegionsWithPBSerialization(c, tableName, regionStartKeys);
}
int createMultiRegionsWithNewHRI(final Configuration c, final HTableDescriptor htd,
final byte[] columnFamily, byte [][] startKeys)
throws IOException {
/**
* Inserts multiple regions into META using PB serialization
*/
int createMultiRegionsWithPBSerialization(final Configuration c, final byte[] tableName,
byte [][] startKeys) throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
if(!htd.hasFamily(columnFamily)) {
HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
htd.addFamily(hcd);
}
List<HRegionInfo> newRegions
= new ArrayList<HRegionInfo>(startKeys.length);
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(htd.getName(),
startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
HRegionInfo hri = new HRegionInfo(tableName, startKeys[i], startKeys[j]);
Put put = MetaEditor.makePutFromRegionInfo(hri);
put.setWriteToWAL(false);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
LOG.info("createMultiRegions: PUT inserted " + hri.toString());
LOG.info("createMultiRegionsWithPBSerialization: PUT inserted " + hri.toString());
newRegions.add(hri);
count++;
@ -364,5 +406,5 @@ public class TestMetaMigrationRemovingHTD {
@org.junit.Rule
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
}
}

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hbase.catalog;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
@ -26,7 +28,14 @@ import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -36,7 +45,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Before;
@ -79,12 +87,41 @@ public class TestMetaReaderEditorNoCluster {
UTIL.shutdownMiniZKCluster();
}
@Test
public void testGetHRegionInfo() throws IOException {
assertNull(HRegionInfo.getHRegionInfo(new Result()));
List<KeyValue> kvs = new ArrayList<KeyValue>();
Result r = new Result(kvs);
assertNull(HRegionInfo.getHRegionInfo(r));
byte [] f = HConstants.CATALOG_FAMILY;
// Make a key value that doesn't have the expected qualifier.
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.SERVER_QUALIFIER, f));
r = new Result(kvs);
assertNull(HRegionInfo.getHRegionInfo(r));
// Make a key that does not have a regioninfo value.
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.REGIONINFO_QUALIFIER, f));
HRegionInfo hri = HRegionInfo.getHRegionInfo(new Result(kvs));
assertTrue(hri == null);
// OK, give it what it expects
kvs.clear();
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.REGIONINFO_QUALIFIER,
HRegionInfo.FIRST_META_REGIONINFO.toByteArray()));
hri = HRegionInfo.getHRegionInfo(new Result(kvs));
assertNotNull(hri);
assertTrue(hri.equals(HRegionInfo.FIRST_META_REGIONINFO));
}
/**
* Test that MetaReader will ride over server throwing
* "Server not running" IOEs.
* @see https://issues.apache.org/jira/browse/HBASE-3446
* @throws IOException
* @throws InterruptedException
* @throws IOException
* @throws InterruptedException
*/
@Test
public void testRideOverServerNotRunning()
@ -112,7 +149,7 @@ public class TestMetaReaderEditorNoCluster {
final byte [] rowToVerify = Bytes.toBytes("rowToVerify");
kvs.add(new KeyValue(rowToVerify,
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(HRegionInfo.FIRST_META_REGIONINFO)));
HRegionInfo.FIRST_META_REGIONINFO.toByteArray()));
kvs.add(new KeyValue(rowToVerify,
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(sn.getHostAndPort())));

View File

@ -19,20 +19,11 @@ package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertNotSame;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.DeserializationException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -42,45 +33,6 @@ import org.apache.zookeeper.KeeperException;
* Package scoped mocking utility.
*/
public class Mocking {
/**
* @param sn ServerName to use making startcode and server in meta
* @param hri Region to serialize into HRegionInfo
* @return A mocked up Result that fakes a Get on a row in the
* <code>.META.</code> table.
* @throws IOException
*/
static Result getMetaTableRowResult(final HRegionInfo hri,
final ServerName sn)
throws IOException {
// TODO: Move to a utilities class. More than one test case can make use
// of this facility.
List<KeyValue> kvs = new ArrayList<KeyValue>();
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri)));
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(sn.getHostAndPort())));
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(sn.getStartcode())));
return new Result(kvs);
}
/**
* @param sn ServerName to use making startcode and server in meta
* @param hri Region to serialize into HRegionInfo
* @return A mocked up Result that fakes a Get on a row in the <code>.META.</code> table.
* @throws IOException
*/
static Result getMetaTableRowResultAsSplitRegion(final HRegionInfo hri, final ServerName sn)
throws IOException {
hri.setOffline(true);
hri.setSplit(true);
return getMetaTableRowResult(hri, sn);
}
static void waitForRegionPendingOpenInRIT(AssignmentManager am, String encodedName)
throws InterruptedException {
@ -110,7 +62,7 @@ public class Mocking {
* @param sn Name of the regionserver doing the 'opening'
* @param hri Region we're 'opening'.
* @throws KeeperException
* @throws DeserializationException
* @throws DeserializationException
*/
static void fakeRegionServerRegionOpenInZK(HMaster master, final ZooKeeperWatcher w,
final ServerName sn, final HRegionInfo hri)
@ -147,7 +99,7 @@ public class Mocking {
* @param region
* @param expectedState
* @return true if region exists and is in expected state
* @throws DeserializationException
* @throws DeserializationException
*/
static boolean verifyRegionState(ZooKeeperWatcher zkw, HRegionInfo region, EventType expectedState)
throws KeeperException, DeserializationException {

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
@ -554,9 +555,9 @@ public class TestAssignmentManager {
Result r = null;
if (splitRegion) {
r = Mocking.getMetaTableRowResultAsSplitRegion(REGIONINFO, SERVERNAME_A);
r = MetaMockingUtil.getMetaTableRowResultAsSplitRegion(REGIONINFO, SERVERNAME_A);
} else {
r = Mocking.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
r = MetaMockingUtil.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
}
ScanResponse.Builder builder = ScanResponse.newBuilder();
@ -918,7 +919,7 @@ public class TestAssignmentManager {
// with an encoded name by doing a Get on .META.
ClientProtocol ri = Mockito.mock(ClientProtocol.class);
// Get a meta row result that has region up on SERVERNAME_A for REGIONINFO
Result r = Mocking.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
Result r = MetaMockingUtil.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
ScanResponse.Builder builder = ScanResponse.newBuilder();
builder.setMoreResults(true);
builder.addResult(ProtobufUtil.toResult(r));

View File

@ -31,12 +31,11 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -77,10 +76,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
MetaEditor.addRegionToMeta(meta, hri);
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
master.assignRegion(hri);

View File

@ -22,16 +22,12 @@ package org.apache.hadoop.hbase.master;
import static org.apache.hadoop.hbase.util.HFileArchiveTestingUtil.assertArchiveEqualToOriginal;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.TreeMap;
@ -46,13 +42,13 @@ 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.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.HConnection;
@ -70,9 +66,7 @@ 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.Pair;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
@ -146,7 +140,7 @@ public class TestCatalogJanitor {
public void abort(String why, Throwable e) {
//no-op
}
@Override
public boolean isAborted() {
return false;
@ -235,7 +229,7 @@ public class TestCatalogJanitor {
public void abort(String why, Throwable e) {
//no-op
}
@Override
public boolean isAborted() {
return false;
@ -261,29 +255,29 @@ public class TestCatalogJanitor {
// TODO Auto-generated method stub
return null;
}
@Override
public Map<String, HTableDescriptor> getAll() throws IOException {
// TODO Auto-generated method stub
return null;
}
@Override
public HTableDescriptor get(byte[] tablename)
throws FileNotFoundException, IOException {
return get(Bytes.toString(tablename));
}
@Override
public HTableDescriptor get(String tablename)
throws FileNotFoundException, IOException {
return createHTableDescriptor();
}
@Override
public void add(HTableDescriptor htd) throws IOException {
// TODO Auto-generated method stub
}
};
}
@ -294,33 +288,6 @@ public class TestCatalogJanitor {
}
}
@Test
public void testGetHRegionInfo() throws IOException {
assertNull(CatalogJanitor.getHRegionInfo(new Result()));
List<KeyValue> kvs = new ArrayList<KeyValue>();
Result r = new Result(kvs);
assertNull(CatalogJanitor.getHRegionInfo(r));
byte [] f = HConstants.CATALOG_FAMILY;
// Make a key value that doesn't have the expected qualifier.
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.SERVER_QUALIFIER, f));
r = new Result(kvs);
assertNull(CatalogJanitor.getHRegionInfo(r));
// Make a key that does not have a regioninfo value.
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.REGIONINFO_QUALIFIER, f));
HRegionInfo hri = CatalogJanitor.getHRegionInfo(new Result(kvs));
assertTrue(hri == null);
// OK, give it what it expects
kvs.clear();
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(HRegionInfo.FIRST_META_REGIONINFO)));
hri = CatalogJanitor.getHRegionInfo(new Result(kvs));
assertNotNull(hri);
assertTrue(hri.equals(HRegionInfo.FIRST_META_REGIONINFO));
}
@Test
public void testCleanParent() throws IOException, InterruptedException {
HBaseTestingUtility htu = new HBaseTestingUtility();
@ -343,12 +310,7 @@ public class TestCatalogJanitor {
Bytes.toBytes("eee"));
// Test that when both daughter regions are in place, that we do not
// remove the parent.
List<KeyValue> kvs = new ArrayList<KeyValue>();
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER, Writables.getBytes(splita)));
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER, Writables.getBytes(splitb)));
Result r = new Result(kvs);
Result r = createResult(parent, splita, splitb);
// Add a reference under splitA directory so we don't clear out the parent.
Path rootdir = services.getMasterFileSystem().getRootDir();
Path tabledir =
@ -541,9 +503,9 @@ public class TestCatalogJanitor {
final Map<HRegionInfo, Result> splitParents =
new TreeMap<HRegionInfo, Result>(new SplitParentFirstComparator());
splitParents.put(parent, makeResultFromHRegionInfo(parent, splita, splitb));
splitParents.put(parent, createResult(parent, splita, splitb));
splita.setOffline(true); //simulate that splita goes offline when it is split
splitParents.put(splita, makeResultFromHRegionInfo(splita, splitaa, splitab));
splitParents.put(splita, createResult(splita, splitaa,splitab));
CatalogJanitor janitor = spy(new CatalogJanitor(server, services));
doReturn(new Pair<Integer, Map<HRegionInfo, Result>>(
@ -586,12 +548,7 @@ public class TestCatalogJanitor {
HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
// Test that when both daughter regions are in place, that we do not
// remove the parent.
List<KeyValue> kvs = new ArrayList<KeyValue>();
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER, Writables.getBytes(splita)));
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER, Writables.getBytes(splitb)));
Result r = new Result(kvs);
Result r = createResult(parent, splita, splitb);
FileSystem fs = FileSystem.get(htu.getConfiguration());
Path rootdir = services.getMasterFileSystem().getRootDir();
@ -651,12 +608,7 @@ public class TestCatalogJanitor {
HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
// Test that when both daughter regions are in place, that we do not
// remove the parent.
List<KeyValue> kvs = new ArrayList<KeyValue>();
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER, Writables.getBytes(splita)));
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER, Writables.getBytes(splitb)));
Result r = new Result(kvs);
Result r = createResult(parent, splita, splitb);
FileSystem fs = FileSystem.get(htu.getConfiguration());
@ -722,31 +674,6 @@ public class TestCatalogJanitor {
assertEquals(count, storeFiles.length);
}
private Result makeResultFromHRegionInfo(HRegionInfo region, HRegionInfo splita,
HRegionInfo splitb) throws IOException {
List<KeyValue> kvs = new ArrayList<KeyValue>();
kvs.add(new KeyValue(
region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(region)));
if (splita != null) {
kvs.add(new KeyValue(
region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
Writables.getBytes(splita)));
}
if (splitb != null) {
kvs.add(new KeyValue(
region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
Writables.getBytes(splitb)));
}
return new Result(kvs);
}
private String setRootDirAndCleanIt(final HBaseTestingUtility htu,
final String subdir)
throws IOException {
@ -788,12 +715,7 @@ public class TestCatalogJanitor {
private Result createResult(final HRegionInfo parent, final HRegionInfo a,
final HRegionInfo b)
throws IOException {
List<KeyValue> kvs = new ArrayList<KeyValue>();
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER, Writables.getBytes(a)));
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER, Writables.getBytes(b)));
return new Result(kvs);
return MetaMockingUtil.getMetaTableRowResult(parent, null, a, b);
}
private HTableDescriptor createHTableDescriptor() {

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
@ -153,17 +154,17 @@ public class TestMasterNoCluster {
RootRegionTracker.setRootLocation(rs0.getZooKeeper(), rs0.getServerName());
byte [] rootregion = Bytes.toBytes("-ROOT-,,0");
rs0.setGetResult(rootregion, HRegionInfo.FIRST_META_REGIONINFO.getRegionName(),
Mocking.getMetaTableRowResult(HRegionInfo.FIRST_META_REGIONINFO,
MetaMockingUtil.getMetaTableRowResult(HRegionInfo.FIRST_META_REGIONINFO,
rs1.getServerName()));
final byte [] tableName = Bytes.toBytes("t");
Result [] results = new Result [] {
Mocking.getMetaTableRowResult(
MetaMockingUtil.getMetaTableRowResult(
new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HBaseTestingUtility.KEYS[1]),
rs2.getServerName()),
Mocking.getMetaTableRowResult(
MetaMockingUtil.getMetaTableRowResult(
new HRegionInfo(tableName, HBaseTestingUtility.KEYS[1], HBaseTestingUtility.KEYS[2]),
rs2.getServerName()),
Mocking.getMetaTableRowResult(new HRegionInfo(tableName, HBaseTestingUtility.KEYS[2],
MetaMockingUtil.getMetaTableRowResult(new HRegionInfo(tableName, HBaseTestingUtility.KEYS[2],
HConstants.EMPTY_END_ROW),
rs2.getServerName())
};
@ -349,7 +350,7 @@ public class TestMasterNoCluster {
// confirm .META. has a server.
byte [] rootregion = Bytes.toBytes("-ROOT-,,0");
rs0.setGetResult(rootregion, HRegionInfo.FIRST_META_REGIONINFO.getRegionName(),
Mocking.getMetaTableRowResult(HRegionInfo.FIRST_META_REGIONINFO,
MetaMockingUtil.getMetaTableRowResult(HRegionInfo.FIRST_META_REGIONINFO,
rs0.getServerName()));
// Master should now come up.
while (!master.isInitialized()) {Threads.sleep(10);}

View File

@ -23,14 +23,16 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
@ -97,7 +99,7 @@ public class TestMasterTransitions {
private int closeCount = 0;
static final int SERVER_DURATION = 3 * 1000;
static final int CLOSE_DURATION = 1 * 1000;
HBase2428Listener(final MiniHBaseCluster c, final HServerAddress metaAddress,
final HRegionInfo closingHRI, final int otherServerIndex) {
this.cluster = c;
@ -183,7 +185,7 @@ public class TestMasterTransitions {
/**
* In 2428, the meta region has just been set offline and then a close comes
* in.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-2428">HBASE-2428</a>
* @see <a href="https://issues.apache.org/jira/browse/HBASE-2428">HBASE-2428</a>
*/
@Ignore @Test (timeout=300000) public void testRegionCloseWhenNoMetaHBase2428()
throws Exception {
@ -205,7 +207,7 @@ public class TestMasterTransitions {
// Get a region out on the otherServer.
final HRegionInfo hri =
otherServer.getOnlineRegions().iterator().next().getRegionInfo();
// Add our RegionServerOperationsListener
HBase2428Listener listener = new HBase2428Listener(cluster,
metaHRS.getHServerInfo().getServerAddress(), hri, otherServerIndex);
@ -312,7 +314,7 @@ public class TestMasterTransitions {
this.copyOfOnlineRegions =
this.victim.getCopyOfOnlineRegionsSortedBySize().values();
}
@Override
public boolean process(HServerInfo serverInfo, HMsg incomingMsg) {
if (!victim.getServerInfo().equals(serverInfo) ||
@ -365,7 +367,7 @@ public class TestMasterTransitions {
* we kill it. We then wait on all regions to come back on line. If bug
* is fixed, this should happen soon as the processing of the killed server is
* done.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-2482">HBASE-2482</a>
* @see <a href="https://issues.apache.org/jira/browse/HBASE-2482">HBASE-2482</a>
*/
@Ignore @Test (timeout=300000) public void testKillRSWithOpeningRegion2482()
throws Exception {
@ -483,10 +485,9 @@ public class TestMasterTransitions {
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
ResultScanner s = meta.getScanner(scan);
for (Result r = null; (r = s.next()) != null;) {
byte [] b =
r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (b == null || b.length <= 0) break;
HRegionInfo hri = Writables.getHRegionInfo(b);
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri == null) break;
// If start key, add 'aaa'.
byte [] row = getStartKey(hri);
Put p = new Put(row);

View File

@ -20,6 +20,11 @@
package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.atomic.AtomicBoolean;
@ -27,7 +32,12 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -41,7 +51,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
@ -51,11 +60,6 @@ import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.mockito.internal.util.reflection.Whitebox;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.junit.Assert.assertFalse;
/**
* Test open and close of regions using zk.
*/
@ -265,7 +269,7 @@ public class TestZKBasedOpenCloseRegion {
// remove the block and reset the boolean
hr1.getRegionsInTransitionInRS().remove(hri.getEncodedNameAsBytes());
reopenEventProcessed.set(false);
// now try moving a region when there is no region in transition.
hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr1));
@ -275,7 +279,7 @@ public class TestZKBasedOpenCloseRegion {
cluster.getMaster().executorService.
registerListener(EventType.RS_ZK_REGION_OPENED, openListener);
TEST_UTIL.getHBaseAdmin().move(hri.getEncodedNameAsBytes(),
Bytes.toBytes(hr0.getServerName().toString()));
@ -336,7 +340,7 @@ public class TestZKBasedOpenCloseRegion {
assertFalse("Region should not be in RIT",
regionServer.getRegionsInTransitionInRS().containsKey(REGIONINFO.getEncodedNameAsBytes()));
}
private static void waitUntilAllRegionsAssigned()
throws IOException {
HTable meta = new HTable(TEST_UTIL.getConfiguration(),
@ -381,12 +385,9 @@ public class TestZKBasedOpenCloseRegion {
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
ResultScanner s = meta.getScanner(scan);
for (Result r = null; (r = s.next()) != null;) {
byte [] b =
r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (b == null || b.length <= 0) {
break;
}
HRegionInfo hri = Writables.getHRegionInfo(b);
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri == null) break;
// If start key, add 'aaa'.
byte [] row = getStartKey(hri);
Put p = new Put(row);

View File

@ -1,62 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.migration;
import java.io.IOException;
import junit.framework.Assert;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD;
import org.apache.hadoop.hbase.util.Writables;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Migration tests that do not need spin up of a cluster.
* @deprecated Remove after we release 0.92
*/
@Category(SmallTests.class)
public class TestMigrationFrom090To092 {
@Test
public void testMigrateHRegionInfoFromVersion0toVersion1()
throws IOException {
HTableDescriptor htd =
getHTableDescriptor("testMigrateHRegionInfoFromVersion0toVersion1");
HRegionInfo090x ninety =
new HRegionInfo090x(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
byte [] bytes = Writables.getBytes(ninety);
// Now deserialize into an HRegionInfo
HRegionInfo hri = Writables.getHRegionInfo(bytes);
Assert.assertEquals(hri.getTableNameAsString(),
ninety.getTableDesc().getNameAsString());
Assert.assertEquals(HRegionInfo.VERSION, hri.getVersion());
}
private HTableDescriptor getHTableDescriptor(final String name) {
HTableDescriptor htd = new HTableDescriptor(name);
htd.addFamily(new HColumnDescriptor("family"));
return htd;
}
@org.junit.Rule
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
}

View File

@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
@ -328,10 +327,10 @@ public class TestEndToEndSplitTransaction {
break;
}
HRegionInfo region = MetaEditor.getHRegionInfo(result);
HRegionInfo region = HRegionInfo.getHRegionInfo(result);
if(region.isSplitParent()) {
log("found parent region: " + region.toString());
PairOfSameType<HRegionInfo> pair = MetaEditor.getDaughterRegions(result);
PairOfSameType<HRegionInfo> pair = HRegionInfo.getDaughterRegions(result);
daughterA = pair.getFirst();
daughterB = pair.getSecond();
break;
@ -367,7 +366,7 @@ public class TestEndToEndSplitTransaction {
while (System.currentTimeMillis() - start < timeout) {
Result result = getRegionRow(regionName);
if (result != null) {
HRegionInfo info = MetaEditor.getHRegionInfo(result);
HRegionInfo info = HRegionInfo.getHRegionInfo(result);
if (info != null && !info.isOffline()) {
log("found region in META: " + Bytes.toStringBinary(regionName));
break;

View File

@ -27,14 +27,19 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseTestCase;
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.MediumTests;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.TestGet;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.experimental.categories.Category;
/**
@ -75,10 +80,9 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
HRegionInfo hri = new HRegionInfo(htd.getName(),
i == 0? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i),
i == last? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i + interval));
Put put = new Put(hri.getRegionName());
Put put = MetaEditor.makePutFromRegionInfo(hri);
put.setWriteToWAL(false);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
mr.put(put, false);
}
}

View File

@ -19,8 +19,6 @@
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@ -46,7 +44,6 @@ import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.junit.experimental.categories.Category;
/**
@ -241,11 +238,8 @@ public class TestScanner extends HBaseTestCase {
Put put = new Put(ROW_KEY, System.currentTimeMillis(), null);
ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
DataOutputStream s = new DataOutputStream(byteStream);
REGION_INFO.write(s);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
byteStream.toByteArray());
REGION_INFO.toByteArray());
region.put(put);
// What we just committed is in the memstore. Verify that we can get
@ -346,8 +340,7 @@ public class TestScanner extends HBaseTestCase {
/** Compare the HRegionInfo we read from HBase to what we stored */
private void validateRegionInfo(byte [] regionBytes) throws IOException {
HRegionInfo info =
(HRegionInfo) Writables.getWritable(regionBytes, new HRegionInfo());
HRegionInfo info = HRegionInfo.parseFromOrNull(regionBytes);
assertEquals(REGION_INFO.getRegionId(), info.getRegionId());
assertEquals(0, info.getStartKey().length);
@ -489,7 +482,7 @@ public class TestScanner extends HBaseTestCase {
/**
* Make sure scanner returns correct result when we run a major compaction
* with deletes.
*
*
* @throws Exception
*/
@SuppressWarnings("deprecation")

View File

@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@ -128,7 +129,7 @@ public class TestHBaseFsck {
@Test
public void testHBaseFsck() throws Exception {
assertNoErrors(doFsck(conf, false));
String table = "tableBadMetaAssign";
String table = "tableBadMetaAssign";
TEST_UTIL.createTable(Bytes.toBytes(table), FAM);
// We created 1 table, should be fine
@ -193,10 +194,8 @@ public class TestHBaseFsck {
throws IOException {
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(htd.getName(), startKey, endKey);
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
MetaEditor.addRegionToMeta(meta, hri);
meta.close();
return hri;
}
@ -300,7 +299,7 @@ public class TestHBaseFsck {
/**
* Setup a clean table before we start mucking with it.
*
*
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
@ -350,7 +349,7 @@ public class TestHBaseFsck {
/**
* delete table in preparation for next test
*
*
* @param tablename
* @throws IOException
*/
@ -406,13 +405,13 @@ public class TestHBaseFsck {
// limit number of threads to 1.
Configuration newconf = new Configuration(conf);
newconf.setInt("hbasefsck.numthreads", 1);
newconf.setInt("hbasefsck.numthreads", 1);
assertNoErrors(doFsck(newconf, false));
// We should pass without triggering a RejectedExecutionException
} finally {
deleteTable(table);
}
}
}
/**
@ -1174,16 +1173,11 @@ public class TestHBaseFsck {
Bytes.toBytes("B"), Bytes.toBytes("BM"));
HRegionInfo b = new HRegionInfo(tbl.getTableName(),
Bytes.toBytes("BM"), Bytes.toBytes("C"));
Put p = new Put(hri.getRegionName());
hri.setOffline(true);
hri.setSplit(true);
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
p.add(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
Writables.getBytes(a));
p.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
Writables.getBytes(b));
meta.put(p);
MetaEditor.addRegionToMeta(meta, hri, a, b);
meta.flushCommits();
TEST_UTIL.getHBaseAdmin().flush(HConstants.META_TABLE_NAME);
@ -1255,7 +1249,7 @@ public class TestHBaseFsck {
deleteTable(table);
}
}
/**
* This creates and fixes a bad table with missing last region -- hole in meta and data missing in
* the fs.

View File

@ -31,7 +31,14 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
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.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -42,7 +49,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
@ -52,10 +58,10 @@ import org.junit.experimental.categories.Category;
* This testing base class creates a minicluster and testing table table
* and shuts down the cluster afterwards. It also provides methods wipes out
* meta and to inject errors into meta and the file system.
*
*
* Tests should generally break stuff, then attempt to rebuild the meta table
* offline, then restart hbase, and finally perform checks.
*
*
* NOTE: This is a slow set of tests which takes ~30s each needs to run on a
* relatively beefy machine. It seems necessary to have each test in a new jvm
* since minicluster startup and tear downs seem to leak file handles and
@ -107,7 +113,7 @@ public class OfflineMetaRebuildTestCore {
/**
* Setup a clean table before we start mucking with it.
*
*
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
@ -142,7 +148,7 @@ public class OfflineMetaRebuildTestCore {
/**
* delete table in preparation for next test
*
*
* @param tablename
* @throws IOException
*/
@ -211,11 +217,8 @@ public class OfflineMetaRebuildTestCore {
out.close();
// add to meta.
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
meta.flushCommits();
MetaEditor.addRegionToMeta(meta, hri);
meta.close();
return hri;
}
@ -240,7 +243,7 @@ public class OfflineMetaRebuildTestCore {
/**
* Returns the number of rows in a given table. HBase must be up and the table
* should be present (will wait for timeout for a while otherwise)
*
*
* @return # of rows in the specified table
*/
protected int tableRowCount(Configuration conf, String table)
@ -259,7 +262,7 @@ public class OfflineMetaRebuildTestCore {
/**
* Dumps .META. table info
*
*
* @return # of entries in meta.
*/
protected int scanMeta() throws IOException {