Switch on-disk formats (reference files, HFile meta fields, etc) to PB
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1340232 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
eef80d6ff9
commit
373270dd77
|
@ -0,0 +1,95 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* The identifier for this cluster.
|
||||
* It is serialized to the filesystem and up into zookeeper. This is a container for the id.
|
||||
* Also knows how to serialize and deserialize the cluster id.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ClusterId {
|
||||
private final String id;
|
||||
|
||||
/**
|
||||
* New ClusterID. Generates a uniqueid.
|
||||
*/
|
||||
public ClusterId() {
|
||||
this(UUID.randomUUID().toString());
|
||||
}
|
||||
|
||||
ClusterId(final String uuid) {
|
||||
this.id = uuid;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The clusterid serialized using pb w/ pb magic prefix
|
||||
*/
|
||||
public byte [] toByteArray() {
|
||||
return ProtobufUtil.prependPBMagic(convert().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes A pb serialized {@link ClusterId} instance with pb magic prefix
|
||||
* @return An instance of {@link ClusterId} made from <code>bytes</code>
|
||||
* @throws DeserializationException
|
||||
* @see {@link #toByteArray()}
|
||||
*/
|
||||
public static ClusterId parseFrom(final byte [] bytes) throws DeserializationException {
|
||||
if (ProtobufUtil.isPBMagicPrefix(bytes)) {
|
||||
int pblen = ProtobufUtil.lengthOfPBMagic();
|
||||
ClusterIdProtos.ClusterId.Builder builder = ClusterIdProtos.ClusterId.newBuilder();
|
||||
ClusterIdProtos.ClusterId cid = null;
|
||||
try {
|
||||
cid = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
return convert(cid);
|
||||
} else {
|
||||
// Presume it was written out this way, the old way.
|
||||
return new ClusterId(Bytes.toString(bytes));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A pb instance to represent this instance.
|
||||
*/
|
||||
ClusterIdProtos.ClusterId convert() {
|
||||
ClusterIdProtos.ClusterId.Builder builder = ClusterIdProtos.ClusterId.newBuilder();
|
||||
return builder.setClusterId(this.id).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param cid
|
||||
* @return A {@link ClusterId} made from the passed in <code>cid</code>
|
||||
*/
|
||||
static ClusterId convert(final ClusterIdProtos.ClusterId cid) {
|
||||
return new ClusterId(cid.getClusterId());
|
||||
}
|
||||
}
|
|
@ -34,6 +34,8 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
|||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.hfile.Compression;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -41,6 +43,8 @@ import org.apache.hadoop.io.Text;
|
|||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* An HColumnDescriptor contains information about a column family such as the
|
||||
|
@ -172,15 +176,15 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
|
|||
DEFAULT_VALUES.put(HConstants.IN_MEMORY, String.valueOf(DEFAULT_IN_MEMORY));
|
||||
DEFAULT_VALUES.put(BLOCKCACHE, String.valueOf(DEFAULT_BLOCKCACHE));
|
||||
DEFAULT_VALUES.put(KEEP_DELETED_CELLS, String.valueOf(DEFAULT_KEEP_DELETED));
|
||||
DEFAULT_VALUES.put(ENCODE_ON_DISK,
|
||||
String.valueOf(DEFAULT_ENCODE_ON_DISK));
|
||||
DEFAULT_VALUES.put(DATA_BLOCK_ENCODING,
|
||||
String.valueOf(DEFAULT_DATA_BLOCK_ENCODING));
|
||||
DEFAULT_VALUES.put(ENCODE_ON_DISK, String.valueOf(DEFAULT_ENCODE_ON_DISK));
|
||||
DEFAULT_VALUES.put(DATA_BLOCK_ENCODING, String.valueOf(DEFAULT_DATA_BLOCK_ENCODING));
|
||||
for (String s : DEFAULT_VALUES.keySet()) {
|
||||
RESERVED_KEYWORDS.add(new ImmutableBytesWritable(Bytes.toBytes(s)));
|
||||
}
|
||||
}
|
||||
|
||||
private static final int UNINITIALIZED = -1;
|
||||
|
||||
// Column family name
|
||||
private byte [] name;
|
||||
|
||||
|
@ -191,11 +195,15 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
|
|||
/*
|
||||
* Cache the max versions rather than calculate it every time.
|
||||
*/
|
||||
private int cachedMaxVersions = -1;
|
||||
private int cachedMaxVersions = UNINITIALIZED;
|
||||
|
||||
/**
|
||||
* Default constructor. Must be present for Writable.
|
||||
* @deprecated Used by Writables and Writables are going away.
|
||||
*/
|
||||
@Deprecated
|
||||
// Make this private rather than remove after deprecation period elapses. Its needed by pb
|
||||
// deserializations.
|
||||
public HColumnDescriptor() {
|
||||
this.name = null;
|
||||
}
|
||||
|
@ -499,6 +507,10 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
|
|||
|
||||
/** @return maximum number of versions */
|
||||
public int getMaxVersions() {
|
||||
if (this.cachedMaxVersions == UNINITIALIZED) {
|
||||
String v = getValue(HConstants.VERSIONS);
|
||||
this.cachedMaxVersions = Integer.parseInt(v);
|
||||
}
|
||||
return this.cachedMaxVersions;
|
||||
}
|
||||
|
||||
|
@ -877,8 +889,10 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
|
|||
return result;
|
||||
}
|
||||
|
||||
// Writable
|
||||
|
||||
/**
|
||||
* @deprecated Writables are going away. Use pb {@link #parseFrom(byte[])} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
int version = in.readByte();
|
||||
if (version < 6) {
|
||||
|
@ -945,6 +959,10 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Writables are going away. Use {@link #toByteArray()} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeByte(COLUMN_DESCRIPTOR_VERSION);
|
||||
Bytes.writeByteArray(out, this.name);
|
||||
|
@ -970,4 +988,62 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
|
|||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return This instance serialized with pb with pb magic prefix
|
||||
* @see {@link #parseFrom(byte[])}
|
||||
*/
|
||||
public byte [] toByteArray() {
|
||||
return ProtobufUtil.prependPBMagic(convert().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes A pb serialized {@link HColumnDescriptor} instance with pb magic prefix
|
||||
* @return An instance of {@link HColumnDescriptor} made from <code>bytes</code>
|
||||
* @throws DeserializationException
|
||||
* @see {@link #toByteArray()}
|
||||
*/
|
||||
public static HColumnDescriptor parseFrom(final byte [] bytes) throws DeserializationException {
|
||||
if (!ProtobufUtil.isPBMagicPrefix(bytes)) throw new DeserializationException("No magic");
|
||||
int pblen = ProtobufUtil.lengthOfPBMagic();
|
||||
ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
|
||||
ColumnFamilySchema cfs = null;
|
||||
try {
|
||||
cfs = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
return convert(cfs);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param cfs
|
||||
* @return An {@link HColumnDescriptor} made from the passed in <code>cfs</code>
|
||||
*/
|
||||
static HColumnDescriptor convert(final ColumnFamilySchema cfs) {
|
||||
// Use the empty constructor so we preserve the initial values set on construction for things
|
||||
// like maxVersion. Otherwise, we pick up wrong values on deserialization which makes for
|
||||
// unrelated-looking test failures that are hard to trace back to here.
|
||||
HColumnDescriptor hcd = new HColumnDescriptor();
|
||||
hcd.name = cfs.getName().toByteArray();
|
||||
for (ColumnFamilySchema.Attribute a: cfs.getAttributesList()) {
|
||||
hcd.setValue(a.getName().toByteArray(), a.getValue().toByteArray());
|
||||
}
|
||||
return hcd;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Convert this instance to a the pb column family type
|
||||
*/
|
||||
ColumnFamilySchema convert() {
|
||||
ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
|
||||
builder.setName(ByteString.copyFrom(getName()));
|
||||
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
|
||||
ColumnFamilySchema.Attribute.Builder aBuilder = ColumnFamilySchema.Attribute.newBuilder();
|
||||
aBuilder.setName(ByteString.copyFrom(e.getKey().get()));
|
||||
aBuilder.setValue(ByteString.copyFrom(e.getValue().get()));
|
||||
builder.addAttributes(aBuilder.build());
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
}
|
|
@ -19,10 +19,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -30,21 +32,28 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
|
||||
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 com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* HRegion information.
|
||||
* Contains HRegion id, start and end keys, a reference to this
|
||||
* HRegions' table descriptor, etc.
|
||||
* Contains HRegion id, start and end keys, a reference to this HRegions' table descriptor, etc.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
|
@ -194,7 +203,10 @@ implements WritableComparable<HRegionInfo> {
|
|||
setHashCode();
|
||||
}
|
||||
|
||||
/** Default constructor - creates empty object */
|
||||
/** Default constructor - creates empty object
|
||||
* @deprecated Used by Writables and Writables are going away.
|
||||
*/
|
||||
@Deprecated
|
||||
public HRegionInfo() {
|
||||
super();
|
||||
}
|
||||
|
@ -229,8 +241,7 @@ implements WritableComparable<HRegionInfo> {
|
|||
* @param endKey end of key range
|
||||
* @throws IllegalArgumentException
|
||||
*/
|
||||
public HRegionInfo(final byte[] tableName, final byte[] startKey,
|
||||
final byte[] endKey)
|
||||
public HRegionInfo(final byte[] tableName, final byte[] startKey, final byte[] endKey)
|
||||
throws IllegalArgumentException {
|
||||
this(tableName, startKey, endKey, false);
|
||||
}
|
||||
|
@ -246,8 +257,8 @@ implements WritableComparable<HRegionInfo> {
|
|||
* regions that may or may not hold references to this region.
|
||||
* @throws IllegalArgumentException
|
||||
*/
|
||||
public HRegionInfo(final byte[] tableName, final byte[] startKey,
|
||||
final byte[] endKey, final boolean split)
|
||||
public HRegionInfo(final byte[] tableName, final byte[] startKey, final byte[] endKey,
|
||||
final boolean split)
|
||||
throws IllegalArgumentException {
|
||||
this(tableName, startKey, endKey, split, System.currentTimeMillis());
|
||||
}
|
||||
|
@ -700,10 +711,11 @@ implements WritableComparable<HRegionInfo> {
|
|||
return VERSION;
|
||||
}
|
||||
|
||||
//
|
||||
// Writable
|
||||
//
|
||||
|
||||
/**
|
||||
* @deprecated Use protobuf serialization instead. See {@link #toByteArray()} and
|
||||
* {@link #toDelimitedByteArray()}
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
super.write(out);
|
||||
|
@ -717,6 +729,11 @@ implements WritableComparable<HRegionInfo> {
|
|||
out.writeInt(hashCode);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use protobuf deserialization instead. See {@link #parseFrom(byte[])} and
|
||||
* {@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
|
||||
|
@ -814,4 +831,156 @@ implements WritableComparable<HRegionInfo> {
|
|||
return isRootRegion()? KeyValue.ROOT_COMPARATOR: isMetaRegion()?
|
||||
KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a HRegionInfo to a RegionInfo
|
||||
*
|
||||
* @param info the HRegionInfo to convert
|
||||
* @return the converted RegionInfo
|
||||
*/
|
||||
RegionInfo convert() {
|
||||
return convert(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a HRegionInfo to a RegionInfo
|
||||
*
|
||||
* @param info the HRegionInfo to convert
|
||||
* @return the converted RegionInfo
|
||||
*/
|
||||
public static RegionInfo convert(final HRegionInfo info) {
|
||||
if (info == null) return null;
|
||||
RegionInfo.Builder builder = RegionInfo.newBuilder();
|
||||
builder.setTableName(ByteString.copyFrom(info.getTableName()));
|
||||
builder.setRegionId(info.getRegionId());
|
||||
if (info.getStartKey() != null) {
|
||||
builder.setStartKey(ByteString.copyFrom(info.getStartKey()));
|
||||
}
|
||||
if (info.getEndKey() != null) {
|
||||
builder.setEndKey(ByteString.copyFrom(info.getEndKey()));
|
||||
}
|
||||
builder.setOffline(info.isOffline());
|
||||
builder.setSplit(info.isSplit());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a RegionInfo to a HRegionInfo
|
||||
*
|
||||
* @param proto the RegionInfo to convert
|
||||
* @return the converted HRegionInfo
|
||||
*/
|
||||
public static HRegionInfo convert(final RegionInfo proto) {
|
||||
if (proto == null) return null;
|
||||
byte [] tableName = proto.getTableName().toByteArray();
|
||||
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
|
||||
return ROOT_REGIONINFO;
|
||||
} else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
|
||||
return FIRST_META_REGIONINFO;
|
||||
}
|
||||
long regionId = proto.getRegionId();
|
||||
byte[] startKey = null;
|
||||
byte[] endKey = null;
|
||||
if (proto.hasStartKey()) {
|
||||
startKey = proto.getStartKey().toByteArray();
|
||||
}
|
||||
if (proto.hasEndKey()) {
|
||||
endKey = proto.getEndKey().toByteArray();
|
||||
}
|
||||
boolean split = false;
|
||||
if (proto.hasSplit()) {
|
||||
split = proto.getSplit();
|
||||
}
|
||||
HRegionInfo hri = new HRegionInfo(tableName, startKey, endKey, split, regionId);
|
||||
if (proto.hasOffline()) {
|
||||
hri.setOffline(proto.getOffline());
|
||||
}
|
||||
return hri;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return This instance serialized as protobuf w/ a magic pb prefix.
|
||||
* @see #parseFrom(byte[]);
|
||||
*/
|
||||
public byte [] toByteArray() {
|
||||
byte [] bytes = convert().toByteArray();
|
||||
return ProtobufUtil.prependPBMagic(bytes);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes
|
||||
* @return A deserialized {@link HRegionInfo} or null if we failed deserialize or passed bytes null
|
||||
* @see {@link #toByteArray()}
|
||||
*/
|
||||
public static HRegionInfo parseFromOrNull(final byte [] bytes) {
|
||||
if (bytes == null || bytes.length <= 0) return null;
|
||||
try {
|
||||
return parseFrom(bytes);
|
||||
} catch (DeserializationException e) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes A pb RegionInfo serialized with a pb magic prefix.
|
||||
* @return
|
||||
* @throws DeserializationException
|
||||
* @see {@link #toByteArray()}
|
||||
*/
|
||||
public static HRegionInfo parseFrom(final byte [] bytes) throws DeserializationException {
|
||||
if (ProtobufUtil.isPBMagicPrefix(bytes)) {
|
||||
int pblen = ProtobufUtil.lengthOfPBMagic();
|
||||
try {
|
||||
HBaseProtos.RegionInfo ri =
|
||||
HBaseProtos.RegionInfo.newBuilder().mergeFrom(bytes, pblen, bytes.length - pblen).build();
|
||||
return convert(ri);
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
return (HRegionInfo)Writables.getWritable(bytes, new HRegionInfo());
|
||||
} catch (IOException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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).
|
||||
* @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
|
||||
* @throws IOException
|
||||
* @see {@link #toByteArray()}
|
||||
*/
|
||||
public byte [] toDelimitedByteArray() throws IOException {
|
||||
return ProtobufUtil.toDelimitedByteArray(convert());
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses an HRegionInfo instance from the passed in stream. Presumes the HRegionInfo was
|
||||
* serialized to the stream with {@link #toDelimitedByteArray()}
|
||||
* @param in
|
||||
* @return An instance of HRegionInfo.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HRegionInfo parseFrom(final FSDataInputStream 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 (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
|
||||
if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
|
||||
return convert(HBaseProtos.RegionInfo.parseDelimitedFrom(is));
|
||||
} 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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -27,6 +27,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
@ -37,12 +38,17 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.io.hfile.Compression;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* HTableDescriptor contains the details about an HBase table such as the descriptors of
|
||||
* all the column families, is the table a catalog table, <code> -ROOT- </code> or
|
||||
|
@ -73,8 +79,6 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
protected final Map<ImmutableBytesWritable, ImmutableBytesWritable> values =
|
||||
new HashMap<ImmutableBytesWritable, ImmutableBytesWritable>();
|
||||
|
||||
private static final String FAMILIES = "FAMILIES";
|
||||
|
||||
public static final String SPLIT_POLICY = "SPLIT_POLICY";
|
||||
|
||||
/**
|
||||
|
@ -185,9 +189,18 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
RESERVED_KEYWORDS.add(IS_META_KEY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Cache of whether this is a meta table or not.
|
||||
*/
|
||||
private volatile Boolean meta = null;
|
||||
/**
|
||||
* Cache of whether this is root table or not.
|
||||
*/
|
||||
private volatile Boolean root = null;
|
||||
private Boolean isDeferredLog = null;
|
||||
/**
|
||||
* Cache of whether deferred logging set.
|
||||
*/
|
||||
private Boolean deferredLog = null;
|
||||
|
||||
/**
|
||||
* Maps column family name to the respective HColumnDescriptors
|
||||
|
@ -230,7 +243,9 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
* Default constructor which constructs an empty object.
|
||||
* For deserializing an HTableDescriptor instance only.
|
||||
* @see #HTableDescriptor(byte[])
|
||||
* @deprecated Used by Writables and Writables are going away.
|
||||
*/
|
||||
@Deprecated
|
||||
public HTableDescriptor() {
|
||||
super();
|
||||
}
|
||||
|
@ -548,11 +563,11 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
* @see #setDeferredLogFlush(boolean)
|
||||
*/
|
||||
public synchronized boolean isDeferredLogFlush() {
|
||||
if(this.isDeferredLog == null) {
|
||||
this.isDeferredLog =
|
||||
if(this.deferredLog == null) {
|
||||
this.deferredLog =
|
||||
isSomething(DEFERRED_LOG_FLUSH_KEY, DEFAULT_DEFERRED_LOG_FLUSH);
|
||||
}
|
||||
return this.isDeferredLog;
|
||||
return this.deferredLog;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -571,7 +586,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
*/
|
||||
public void setDeferredLogFlush(final boolean isDeferredLogFlush) {
|
||||
setValue(DEFERRED_LOG_FLUSH_KEY, isDeferredLogFlush? TRUE: FALSE);
|
||||
this.isDeferredLog = isDeferredLogFlush;
|
||||
this.deferredLog = isDeferredLogFlush;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -836,11 +851,12 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
return result;
|
||||
}
|
||||
|
||||
// Writable
|
||||
/**
|
||||
* <em> INTERNAL </em> This method is a part of {@link WritableComparable} interface
|
||||
* and is used for de-serialization of the HTableDescriptor over RPC
|
||||
* @deprecated Writables are going away. Use pb {@link #parseFrom(byte[])} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
int version = in.readInt();
|
||||
|
@ -875,7 +891,9 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
/**
|
||||
* <em> INTERNAL </em> This method is a part of {@link WritableComparable} interface
|
||||
* and is used for serialization of the HTableDescriptor over RPC
|
||||
* @deprecated Writables are going away. Use pb {@link #toByteArray()(byte[])} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeInt(TABLE_DESCRIPTOR_VERSION);
|
||||
|
@ -1198,4 +1216,71 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
// .META. and -ROOT- should return system user as owner, not null (see MasterFileSystem.java:bootstrap()).
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return This instance serialized with pb with pb magic prefix
|
||||
* @see {@link #parseFrom(byte[])}
|
||||
*/
|
||||
public byte [] toByteArray() {
|
||||
return ProtobufUtil.prependPBMagic(convert().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes A pb serialized {@link HTableDescriptor} instance with pb magic prefix
|
||||
* @return An instance of {@link HTableDescriptor} made from <code>bytes</code>
|
||||
* @throws DeserializationException
|
||||
* @throws IOException
|
||||
* @see {@link #toByteArray()}
|
||||
*/
|
||||
public static HTableDescriptor parseFrom(final byte [] bytes)
|
||||
throws DeserializationException, IOException {
|
||||
if (!ProtobufUtil.isPBMagicPrefix(bytes)) {
|
||||
return (HTableDescriptor)Writables.getWritable(bytes, new HTableDescriptor());
|
||||
}
|
||||
int pblen = ProtobufUtil.lengthOfPBMagic();
|
||||
TableSchema.Builder builder = TableSchema.newBuilder();
|
||||
TableSchema ts = null;
|
||||
try {
|
||||
ts = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
return convert(ts);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Convert the current {@link HTableDescriptor} into a pb TableSchema instance.
|
||||
*/
|
||||
TableSchema convert() {
|
||||
TableSchema.Builder builder = TableSchema.newBuilder();
|
||||
builder.setName(ByteString.copyFrom(getName()));
|
||||
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
|
||||
TableSchema.Attribute.Builder aBuilder = TableSchema.Attribute.newBuilder();
|
||||
aBuilder.setName(ByteString.copyFrom(e.getKey().get()));
|
||||
aBuilder.setValue(ByteString.copyFrom(e.getValue().get()));
|
||||
builder.addAttributes(aBuilder.build());
|
||||
}
|
||||
for (HColumnDescriptor hcd: getColumnFamilies()) {
|
||||
builder.addColumnFamilies(hcd.convert());
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param ts A pb TableSchema instance.
|
||||
* @return An {@link HTableDescriptor} made from the passed in pb <code>ts</code>.
|
||||
*/
|
||||
static HTableDescriptor convert(final TableSchema ts) {
|
||||
List<ColumnFamilySchema> list = ts.getColumnFamiliesList();
|
||||
HColumnDescriptor [] hcds = new HColumnDescriptor[list.size()];
|
||||
int index = 0;
|
||||
for (ColumnFamilySchema cfs: list) {
|
||||
hcds[index++] = HColumnDescriptor.convert(cfs);
|
||||
}
|
||||
HTableDescriptor htd = new HTableDescriptor(ts.getName().toByteArray(), hcds);
|
||||
for (TableSchema.Attribute a: ts.getAttributesList()) {
|
||||
htd.setValue(a.getName().toByteArray(), a.getValue().toByteArray());
|
||||
}
|
||||
return htd;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -83,7 +83,7 @@ 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.Writables;
|
||||
import org.apache.hadoop.hbase.zookeeper.ClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTable;
|
||||
|
@ -637,7 +637,7 @@ public class HConnectionManager {
|
|||
ZooKeeperKeepAliveConnection zkw = null;
|
||||
try {
|
||||
zkw = getKeepAliveZooKeeperWatcher();
|
||||
this.clusterId = ClusterId.readClusterIdZNode(zkw);
|
||||
this.clusterId = ZKClusterId.readClusterIdZNode(zkw);
|
||||
if (clusterId == null) {
|
||||
LOG.info("ClusterId read in ZooKeeper is null");
|
||||
}
|
||||
|
|
|
@ -19,9 +19,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.io;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
|
@ -29,16 +31,21 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
/**
|
||||
* A reference to the top or bottom half of a store file. The file referenced
|
||||
* A reference to the top or bottom half of a store file where 'bottom' is the first half
|
||||
* of the file containing the keys that sort lowest and 'top' is the second half
|
||||
* of the file with keys that sort greater than those of the bottom half. The file referenced
|
||||
* lives under a different region. References are made at region split time.
|
||||
*
|
||||
* <p>References work with a special half store file type. References know how
|
||||
* to write out the reference format in the file system and are whats juggled
|
||||
* to write out the reference format in the file system and are what is juggled
|
||||
* when references are mixed in with direct store files. The half store file
|
||||
* type is used reading the referred to file.
|
||||
*
|
||||
|
@ -46,7 +53,7 @@ import org.apache.hadoop.io.Writable;
|
|||
* this in the file system
|
||||
* <code>1278437856009925445.3323223323</code>:
|
||||
* i.e. an id followed by hash of the referenced region.
|
||||
* Note, a region is itself not splitable if it has instances of store file
|
||||
* Note, a region is itself not splittable if it has instances of store file
|
||||
* references. References are cleaned up by compactions.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
|
@ -58,27 +65,45 @@ public class Reference implements Writable {
|
|||
* For split HStoreFiles, it specifies if the file covers the lower half or
|
||||
* the upper half of the key range
|
||||
*/
|
||||
public static enum Range {
|
||||
static enum Range {
|
||||
/** HStoreFile contains upper half of key range */
|
||||
top,
|
||||
/** HStoreFile contains lower half of key range */
|
||||
bottom
|
||||
}
|
||||
|
||||
/**
|
||||
* @param splitRow
|
||||
* @return A {@link Reference} that points at top half of a an hfile
|
||||
*/
|
||||
public static Reference createTopReference(final byte [] splitRow) {
|
||||
return new Reference(splitRow, Range.top);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param splitRow
|
||||
* @return A {@link Reference} that points at the bottom half of a an hfile
|
||||
*/
|
||||
public static Reference createBottomReference(final byte [] splitRow) {
|
||||
return new Reference(splitRow, Range.bottom);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param splitRow This is row we are splitting around.
|
||||
* @param fr
|
||||
*/
|
||||
public Reference(final byte [] splitRow, final Range fr) {
|
||||
this.splitkey = splitRow == null?
|
||||
null: KeyValue.createFirstOnRow(splitRow).getKey();
|
||||
Reference(final byte [] splitRow, final Range fr) {
|
||||
this.splitkey = splitRow == null? null: KeyValue.createFirstOnRow(splitRow).getKey();
|
||||
this.region = fr;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by serializations.
|
||||
* @deprecated Use the pb serializations instead. Writables are going away.
|
||||
*/
|
||||
@Deprecated
|
||||
// Make this private when it comes time to let go of this constructor. Needed by pb serialization.
|
||||
public Reference() {
|
||||
this(null, Range.bottom);
|
||||
}
|
||||
|
@ -106,14 +131,20 @@ public class Reference implements Writable {
|
|||
return "" + this.region;
|
||||
}
|
||||
|
||||
// Make it serializable.
|
||||
|
||||
/**
|
||||
* @deprecated Writables are going away. Use the pb serialization methods instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public void write(DataOutput out) throws IOException {
|
||||
// Write true if we're doing top of the file.
|
||||
out.writeBoolean(isTopFileRegion(this.region));
|
||||
Bytes.writeByteArray(out, this.splitkey);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Writables are going away. Use the pb serialization methods instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
boolean tmp = in.readBoolean();
|
||||
// If true, set region to top.
|
||||
|
@ -129,7 +160,7 @@ public class Reference implements Writable {
|
|||
throws IOException {
|
||||
FSDataOutputStream out = fs.create(p, false);
|
||||
try {
|
||||
write(out);
|
||||
out.write(toByteArray());
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
|
@ -147,11 +178,63 @@ public class Reference implements Writable {
|
|||
throws IOException {
|
||||
FSDataInputStream in = fs.open(p);
|
||||
try {
|
||||
Reference r = new Reference();
|
||||
r.readFields(in);
|
||||
return r;
|
||||
return parseFrom(in);
|
||||
} finally {
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
|
||||
FSProtos.Reference convert() {
|
||||
FSProtos.Reference.Builder builder = FSProtos.Reference.newBuilder();
|
||||
builder.setRange(isTopFileRegion(getFileRegion())?
|
||||
FSProtos.Reference.Range.TOP: FSProtos.Reference.Range.BOTTOM);
|
||||
builder.setSplitkey(ByteString.copyFrom(getSplitKey()));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
static Reference convert(final FSProtos.Reference r) {
|
||||
Reference result = new Reference();
|
||||
result.splitkey = r.getSplitkey().toByteArray();
|
||||
result.region = r.getRange() == FSProtos.Reference.Range.TOP? Range.top: Range.bottom;
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 ou want).
|
||||
* @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
|
||||
* @throws IOException
|
||||
* @see {@link #toByteArray()}
|
||||
*/
|
||||
byte [] toByteArray() throws IOException {
|
||||
return ProtobufUtil.prependPBMagic(convert().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses an {@link Reference} instance from the passed in stream. Presumes the
|
||||
* Reference was serialized to the stream with {@link #toDelimitedByteArray()}
|
||||
* @param in
|
||||
* @return An instance of Reference.
|
||||
* @throws IOException
|
||||
*/
|
||||
static Reference parseFrom(final FSDataInputStream 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 (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
|
||||
if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
|
||||
return convert(FSProtos.Reference.parseFrom(is));
|
||||
} else {
|
||||
// Presume Writables. Need to reset the stream since it didn't start w/ pb.
|
||||
// We won't bother rewriting thie Reference as a pb since Reference is transitory.
|
||||
in.reset();
|
||||
Reference r = new Reference();
|
||||
r.readFields(in);
|
||||
return r;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -64,7 +64,6 @@ import org.apache.hadoop.hbase.client.HTable;
|
|||
import org.apache.hadoop.hbase.client.ServerCallable;
|
||||
import org.apache.hadoop.hbase.io.HalfStoreFileReader;
|
||||
import org.apache.hadoop.hbase.io.Reference;
|
||||
import org.apache.hadoop.hbase.io.Reference.Range;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
|
||||
|
@ -93,10 +92,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class LoadIncrementalHFiles extends Configured implements Tool {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class);
|
||||
private static final int TABLE_CREATE_MAX_RETRIES = 20;
|
||||
private static final long TABLE_CREATE_SLEEP = 60000;
|
||||
static AtomicLong regionCount = new AtomicLong(0);
|
||||
private HBaseAdmin hbAdmin;
|
||||
private Configuration cfg;
|
||||
|
@ -519,8 +515,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
Path bottomOut, Path topOut) throws IOException
|
||||
{
|
||||
// Open reader with no block cache, and not in-memory
|
||||
Reference topReference = new Reference(splitKey, Range.top);
|
||||
Reference bottomReference = new Reference(splitKey, Range.bottom);
|
||||
Reference topReference = Reference.createTopReference(splitKey);
|
||||
Reference bottomReference = Reference.createBottomReference(splitKey);
|
||||
|
||||
copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
|
||||
copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
|
||||
|
|
|
@ -105,7 +105,7 @@ import org.apache.hadoop.hbase.util.Sleeper;
|
|||
import org.apache.hadoop.hbase.util.Strings;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.util.VersionInfo;
|
||||
import org.apache.hadoop.hbase.zookeeper.ClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
|
||||
|
@ -555,7 +555,7 @@ Server {
|
|||
|
||||
// publish cluster ID
|
||||
status.setStatus("Publishing Cluster ID in ZooKeeper");
|
||||
ClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
|
||||
ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
|
||||
|
||||
this.executorService = new ExecutorService(getServerName().toString());
|
||||
|
||||
|
@ -1524,7 +1524,7 @@ Server {
|
|||
}});
|
||||
|
||||
return new ClusterStatus(VersionInfo.getVersion(),
|
||||
this.fileSystemManager.getClusterId(),
|
||||
this.fileSystemManager.getClusterId().toString(),
|
||||
this.serverManager.getOnlineServers(),
|
||||
this.serverManager.getDeadServers(),
|
||||
this.serverName,
|
||||
|
@ -1534,7 +1534,7 @@ Server {
|
|||
}
|
||||
|
||||
public String getClusterId() {
|
||||
return fileSystemManager.getClusterId();
|
||||
return fileSystemManager.getClusterId().toString();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -34,6 +34,8 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ClusterId;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -67,7 +69,7 @@ public class MasterFileSystem {
|
|||
// metrics for master
|
||||
MasterMetrics metrics;
|
||||
// Persisted unique cluster ID
|
||||
private String clusterId;
|
||||
private ClusterId clusterId;
|
||||
// Keep around for convenience.
|
||||
private final FileSystem fs;
|
||||
// Is the fileystem ok?
|
||||
|
@ -178,7 +180,7 @@ public class MasterFileSystem {
|
|||
/**
|
||||
* @return The unique identifier generated for this cluster
|
||||
*/
|
||||
public String getClusterId() {
|
||||
public ClusterId getClusterId() {
|
||||
return clusterId;
|
||||
}
|
||||
|
||||
|
@ -322,8 +324,7 @@ public class MasterFileSystem {
|
|||
final FileSystem fs)
|
||||
throws IOException {
|
||||
// If FS is in safe mode wait till out of it.
|
||||
FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
|
||||
10 * 1000));
|
||||
FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
|
||||
// Filesystem is good. Go ahead and check for hbase.rootdir.
|
||||
try {
|
||||
if (!fs.exists(rd)) {
|
||||
|
@ -347,6 +348,11 @@ public class MasterFileSystem {
|
|||
10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
|
||||
HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
|
||||
}
|
||||
} catch (DeserializationException de) {
|
||||
LOG.fatal("Please fix invalid configuration for " + HConstants.HBASE_DIR, de);
|
||||
IOException ioe = new IOException();
|
||||
ioe.initCause(de);
|
||||
throw ioe;
|
||||
} catch (IllegalArgumentException iae) {
|
||||
LOG.fatal("Please fix invalid configuration for "
|
||||
+ HConstants.HBASE_DIR + " " + rd.toString(), iae);
|
||||
|
@ -355,8 +361,7 @@ public class MasterFileSystem {
|
|||
// Make sure cluster ID exists
|
||||
if (!FSUtils.checkClusterIdExists(fs, rd, c.getInt(
|
||||
HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000))) {
|
||||
FSUtils.setClusterId(fs, rd, UUID.randomUUID().toString(), c.getInt(
|
||||
HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
|
||||
FSUtils.setClusterId(fs, rd, new ClusterId(), c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
|
||||
}
|
||||
clusterId = FSUtils.getClusterId(fs, rd);
|
||||
|
||||
|
|
|
@ -102,8 +102,7 @@ public class CreateTableHandler extends EventHandler {
|
|||
// table in progress. This will introduce a new zookeeper call. Given
|
||||
// createTable isn't a frequent operation, that should be ok.
|
||||
try {
|
||||
if (!this.assignmentManager.getZKTable().checkAndSetEnablingTable(
|
||||
tableName))
|
||||
if (!this.assignmentManager.getZKTable().checkAndSetEnablingTable(tableName))
|
||||
throw new TableExistsException(tableName);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Unable to ensure that the table will be" +
|
||||
|
@ -127,8 +126,7 @@ public class CreateTableHandler extends EventHandler {
|
|||
String tableName = this.hTableDescriptor.getNameAsString();
|
||||
try {
|
||||
LOG.info("Attemping to create the table " + tableName);
|
||||
MasterCoprocessorHost cpHost = ((HMaster) this.server)
|
||||
.getCoprocessorHost();
|
||||
MasterCoprocessorHost cpHost = ((HMaster) this.server).getCoprocessorHost();
|
||||
if (cpHost != null) {
|
||||
cpHost.preCreateTableHandler(this.hTableDescriptor, this.newRegions);
|
||||
}
|
||||
|
|
|
@ -103,6 +103,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
@ -110,6 +111,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
|
@ -162,7 +164,7 @@ public final class ProtobufUtil {
|
|||
* @return True if passed <code>bytes</code> has {@link #PB_MAGIC} for a prefix.
|
||||
*/
|
||||
public static boolean isPBMagicPrefix(final byte [] bytes) {
|
||||
if (bytes == null || bytes.length <= PB_MAGIC.length) return false;
|
||||
if (bytes == null || bytes.length < PB_MAGIC.length) return false;
|
||||
return Bytes.compareTo(PB_MAGIC, 0, PB_MAGIC.length, bytes, 0, PB_MAGIC.length) == 0;
|
||||
}
|
||||
|
||||
|
@ -287,63 +289,6 @@ public final class ProtobufUtil {
|
|||
return new ServerName(hostName, port, startCode);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a RegionInfo to a HRegionInfo
|
||||
*
|
||||
* @param proto the RegionInfo to convert
|
||||
* @return the converted HRegionInfo
|
||||
*/
|
||||
public static HRegionInfo toRegionInfo(final RegionInfo proto) {
|
||||
if (proto == null) return null;
|
||||
byte[] tableName = proto.getTableName().toByteArray();
|
||||
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
|
||||
return HRegionInfo.ROOT_REGIONINFO;
|
||||
} else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
|
||||
return HRegionInfo.FIRST_META_REGIONINFO;
|
||||
}
|
||||
long regionId = proto.getRegionId();
|
||||
byte[] startKey = null;
|
||||
byte[] endKey = null;
|
||||
if (proto.hasStartKey()) {
|
||||
startKey = proto.getStartKey().toByteArray();
|
||||
}
|
||||
if (proto.hasEndKey()) {
|
||||
endKey = proto.getEndKey().toByteArray();
|
||||
}
|
||||
boolean split = false;
|
||||
if (proto.hasSplit()) {
|
||||
split = proto.getSplit();
|
||||
}
|
||||
HRegionInfo hri = new HRegionInfo(tableName,
|
||||
startKey, endKey, split, regionId);
|
||||
if (proto.hasOffline()) {
|
||||
hri.setOffline(proto.getOffline());
|
||||
}
|
||||
return hri;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a HRegionInfo to a RegionInfo
|
||||
*
|
||||
* @param info the HRegionInfo to convert
|
||||
* @return the converted RegionInfo
|
||||
*/
|
||||
public static RegionInfo toRegionInfo(final HRegionInfo info) {
|
||||
if (info == null) return null;
|
||||
RegionInfo.Builder builder = RegionInfo.newBuilder();
|
||||
builder.setTableName(ByteString.copyFrom(info.getTableName()));
|
||||
builder.setRegionId(info.getRegionId());
|
||||
if (info.getStartKey() != null) {
|
||||
builder.setStartKey(ByteString.copyFrom(info.getStartKey()));
|
||||
}
|
||||
if (info.getEndKey() != null) {
|
||||
builder.setEndKey(ByteString.copyFrom(info.getEndKey()));
|
||||
}
|
||||
builder.setOffline(info.isOffline());
|
||||
builder.setSplit(info.isSplit());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a protocol buffer Get to a client Get
|
||||
*
|
||||
|
@ -1234,7 +1179,7 @@ public final class ProtobufUtil {
|
|||
RequestConverter.buildGetRegionInfoRequest(regionName);
|
||||
GetRegionInfoResponse response =
|
||||
admin.getRegionInfo(null, request);
|
||||
return toRegionInfo(response.getRegionInfo());
|
||||
return HRegionInfo.convert(response.getRegionInfo());
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
}
|
||||
|
@ -1349,23 +1294,30 @@ public final class ProtobufUtil {
|
|||
* @return a list of online region info
|
||||
* @throws IOException
|
||||
*/
|
||||
public static List<HRegionInfo> getOnlineRegions(
|
||||
final AdminProtocol admin) throws IOException {
|
||||
public static List<HRegionInfo> getOnlineRegions(final AdminProtocol admin) throws IOException {
|
||||
GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest();
|
||||
List<HRegionInfo> regions = null;
|
||||
GetOnlineRegionResponse response = null;
|
||||
try {
|
||||
GetOnlineRegionResponse response =
|
||||
admin.getOnlineRegion(null, request);
|
||||
regions = new ArrayList<HRegionInfo>();
|
||||
if (response != null) { // it can be null only mockup testing region sever
|
||||
for (RegionInfo regionInfo: response.getRegionInfoList()) {
|
||||
regions.add(toRegionInfo(regionInfo));
|
||||
}
|
||||
}
|
||||
return regions;
|
||||
response = admin.getOnlineRegion(null, request);
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
}
|
||||
return getRegionInfos(response);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of region info from a GetOnlineRegionResponse
|
||||
*
|
||||
* @param proto the GetOnlineRegionResponse
|
||||
* @return the list of region info or null if <code>proto</code> is null
|
||||
*/
|
||||
static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
|
||||
if (proto == null) return null;
|
||||
List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
|
||||
for (RegionInfo regionInfo: proto.getRegionInfoList()) {
|
||||
regionInfos.add(HRegionInfo.convert(regionInfo));
|
||||
}
|
||||
return regionInfos;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1459,4 +1411,16 @@ public final class ProtobufUtil {
|
|||
|
||||
return rl.getReadRequestsCount() + rl.getWriteRequestsCount();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @param m Message to get delimited pb serialization of (with pb magic prefix)
|
||||
*/
|
||||
public static byte [] toDelimitedByteArray(final Message m) throws IOException {
|
||||
// Allocate arbitrary big size so we avoid resizing.
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
|
||||
m.writeDelimitedTo(baos);
|
||||
baos.close();
|
||||
return ProtobufUtil.prependPBMagic(baos.toByteArray());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -579,7 +579,7 @@ public final class RequestConverter {
|
|||
buildOpenRegionRequest(final List<HRegionInfo> regions) {
|
||||
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
|
||||
for (HRegionInfo region: regions) {
|
||||
builder.addRegion(ProtobufUtil.toRegionInfo(region));
|
||||
builder.addRegion(HRegionInfo.convert(region));
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
@ -605,7 +605,7 @@ public final class RequestConverter {
|
|||
public static OpenRegionRequest buildOpenRegionRequest(
|
||||
final HRegionInfo region, final int versionOfOfflineNode) {
|
||||
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
|
||||
builder.addRegion(ProtobufUtil.toRegionInfo(region));
|
||||
builder.addRegion(HRegionInfo.convert(region));
|
||||
if (versionOfOfflineNode >= 0) {
|
||||
builder.setVersionOfOfflineNode(versionOfOfflineNode);
|
||||
}
|
||||
|
|
|
@ -138,14 +138,9 @@ public final class ResponseConverter {
|
|||
* @param proto the GetOnlineRegionResponse
|
||||
* @return the list of region info
|
||||
*/
|
||||
public static List<HRegionInfo> getRegionInfos
|
||||
(final GetOnlineRegionResponse proto) {
|
||||
public static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
|
||||
if (proto == null || proto.getRegionInfoCount() == 0) return null;
|
||||
List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
|
||||
for (RegionInfo regionInfo: proto.getRegionInfoList()) {
|
||||
regionInfos.add(ProtobufUtil.toRegionInfo(regionInfo));
|
||||
}
|
||||
return regionInfos;
|
||||
return ProtobufUtil.getRegionInfos(proto);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -202,7 +197,7 @@ public final class ResponseConverter {
|
|||
final List<HRegionInfo> regions) {
|
||||
GetOnlineRegionResponse.Builder builder = GetOnlineRegionResponse.newBuilder();
|
||||
for (HRegionInfo region: regions) {
|
||||
builder.addRegionInfo(ProtobufUtil.toRegionInfo(region));
|
||||
builder.addRegionInfo(HRegionInfo.convert(region));
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,468 @@
|
|||
// Generated by the protocol buffer compiler. DO NOT EDIT!
|
||||
// source: ClusterId.proto
|
||||
|
||||
package org.apache.hadoop.hbase.protobuf.generated;
|
||||
|
||||
public final class ClusterIdProtos {
|
||||
private ClusterIdProtos() {}
|
||||
public static void registerAllExtensions(
|
||||
com.google.protobuf.ExtensionRegistry registry) {
|
||||
}
|
||||
public interface ClusterIdOrBuilder
|
||||
extends com.google.protobuf.MessageOrBuilder {
|
||||
|
||||
// required string clusterId = 1;
|
||||
boolean hasClusterId();
|
||||
String getClusterId();
|
||||
}
|
||||
public static final class ClusterId extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
implements ClusterIdOrBuilder {
|
||||
// Use ClusterId.newBuilder() to construct.
|
||||
private ClusterId(Builder builder) {
|
||||
super(builder);
|
||||
}
|
||||
private ClusterId(boolean noInit) {}
|
||||
|
||||
private static final ClusterId defaultInstance;
|
||||
public static ClusterId getDefaultInstance() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public ClusterId getDefaultInstanceForType() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.internal_static_ClusterId_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.internal_static_ClusterId_fieldAccessorTable;
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
// required string clusterId = 1;
|
||||
public static final int CLUSTERID_FIELD_NUMBER = 1;
|
||||
private java.lang.Object clusterId_;
|
||||
public boolean hasClusterId() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public String getClusterId() {
|
||||
java.lang.Object ref = clusterId_;
|
||||
if (ref instanceof String) {
|
||||
return (String) ref;
|
||||
} else {
|
||||
com.google.protobuf.ByteString bs =
|
||||
(com.google.protobuf.ByteString) ref;
|
||||
String s = bs.toStringUtf8();
|
||||
if (com.google.protobuf.Internal.isValidUtf8(bs)) {
|
||||
clusterId_ = s;
|
||||
}
|
||||
return s;
|
||||
}
|
||||
}
|
||||
private com.google.protobuf.ByteString getClusterIdBytes() {
|
||||
java.lang.Object ref = clusterId_;
|
||||
if (ref instanceof String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
|
||||
clusterId_ = b;
|
||||
return b;
|
||||
} else {
|
||||
return (com.google.protobuf.ByteString) ref;
|
||||
}
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
clusterId_ = "";
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
byte isInitialized = memoizedIsInitialized;
|
||||
if (isInitialized != -1) return isInitialized == 1;
|
||||
|
||||
if (!hasClusterId()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
memoizedIsInitialized = 1;
|
||||
return true;
|
||||
}
|
||||
|
||||
public void writeTo(com.google.protobuf.CodedOutputStream output)
|
||||
throws java.io.IOException {
|
||||
getSerializedSize();
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
output.writeBytes(1, getClusterIdBytes());
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
private int memoizedSerializedSize = -1;
|
||||
public int getSerializedSize() {
|
||||
int size = memoizedSerializedSize;
|
||||
if (size != -1) return size;
|
||||
|
||||
size = 0;
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBytesSize(1, getClusterIdBytes());
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
}
|
||||
|
||||
private static final long serialVersionUID = 0L;
|
||||
@java.lang.Override
|
||||
protected java.lang.Object writeReplace()
|
||||
throws java.io.ObjectStreamException {
|
||||
return super.writeReplace();
|
||||
}
|
||||
|
||||
@java.lang.Override
|
||||
public boolean equals(final java.lang.Object obj) {
|
||||
if (obj == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId)) {
|
||||
return super.equals(obj);
|
||||
}
|
||||
org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId other = (org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId) obj;
|
||||
|
||||
boolean result = true;
|
||||
result = result && (hasClusterId() == other.hasClusterId());
|
||||
if (hasClusterId()) {
|
||||
result = result && getClusterId()
|
||||
.equals(other.getClusterId());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
}
|
||||
|
||||
@java.lang.Override
|
||||
public int hashCode() {
|
||||
int hash = 41;
|
||||
hash = (19 * hash) + getDescriptorForType().hashCode();
|
||||
if (hasClusterId()) {
|
||||
hash = (37 * hash) + CLUSTERID_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getClusterId().hashCode();
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
return hash;
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
|
||||
com.google.protobuf.ByteString data)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
|
||||
com.google.protobuf.ByteString data,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(byte[] data)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
|
||||
byte[] data,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(java.io.InputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
|
||||
java.io.InputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseDelimitedFrom(java.io.InputStream input)
|
||||
throws java.io.IOException {
|
||||
Builder builder = newBuilder();
|
||||
if (builder.mergeDelimitedFrom(input)) {
|
||||
return builder.buildParsed();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseDelimitedFrom(
|
||||
java.io.InputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
Builder builder = newBuilder();
|
||||
if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
|
||||
return builder.buildParsed();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
|
||||
com.google.protobuf.CodedInputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
|
||||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
|
||||
public static Builder newBuilder() { return Builder.create(); }
|
||||
public Builder newBuilderForType() { return newBuilder(); }
|
||||
public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId prototype) {
|
||||
return newBuilder().mergeFrom(prototype);
|
||||
}
|
||||
public Builder toBuilder() { return newBuilder(this); }
|
||||
|
||||
@java.lang.Override
|
||||
protected Builder newBuilderForType(
|
||||
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
||||
Builder builder = new Builder(parent);
|
||||
return builder;
|
||||
}
|
||||
public static final class Builder extends
|
||||
com.google.protobuf.GeneratedMessage.Builder<Builder>
|
||||
implements org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterIdOrBuilder {
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.internal_static_ClusterId_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.internal_static_ClusterId_fieldAccessorTable;
|
||||
}
|
||||
|
||||
// Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId.newBuilder()
|
||||
private Builder() {
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
|
||||
private Builder(BuilderParent parent) {
|
||||
super(parent);
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
private void maybeForceBuilderInitialization() {
|
||||
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
|
||||
}
|
||||
}
|
||||
private static Builder create() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public Builder clear() {
|
||||
super.clear();
|
||||
clusterId_ = "";
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder clone() {
|
||||
return create().mergeFrom(buildPartial());
|
||||
}
|
||||
|
||||
public com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptorForType() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId.getDescriptor();
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId getDefaultInstanceForType() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId.getDefaultInstance();
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId build() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId buildParsed()
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(
|
||||
result).asInvalidProtocolBufferException();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId buildPartial() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId result = new org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId(this);
|
||||
int from_bitField0_ = bitField0_;
|
||||
int to_bitField0_ = 0;
|
||||
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
to_bitField0_ |= 0x00000001;
|
||||
}
|
||||
result.clusterId_ = clusterId_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
}
|
||||
|
||||
public Builder mergeFrom(com.google.protobuf.Message other) {
|
||||
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId) {
|
||||
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId)other);
|
||||
} else {
|
||||
super.mergeFrom(other);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId other) {
|
||||
if (other == org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId.getDefaultInstance()) return this;
|
||||
if (other.hasClusterId()) {
|
||||
setClusterId(other.getClusterId());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
||||
public final boolean isInitialized() {
|
||||
if (!hasClusterId()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public Builder mergeFrom(
|
||||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
|
||||
com.google.protobuf.UnknownFieldSet.newBuilder(
|
||||
this.getUnknownFields());
|
||||
while (true) {
|
||||
int tag = input.readTag();
|
||||
switch (tag) {
|
||||
case 0:
|
||||
this.setUnknownFields(unknownFields.build());
|
||||
onChanged();
|
||||
return this;
|
||||
default: {
|
||||
if (!parseUnknownField(input, unknownFields,
|
||||
extensionRegistry, tag)) {
|
||||
this.setUnknownFields(unknownFields.build());
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
break;
|
||||
}
|
||||
case 10: {
|
||||
bitField0_ |= 0x00000001;
|
||||
clusterId_ = input.readBytes();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
|
||||
// required string clusterId = 1;
|
||||
private java.lang.Object clusterId_ = "";
|
||||
public boolean hasClusterId() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public String getClusterId() {
|
||||
java.lang.Object ref = clusterId_;
|
||||
if (!(ref instanceof String)) {
|
||||
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
|
||||
clusterId_ = s;
|
||||
return s;
|
||||
} else {
|
||||
return (String) ref;
|
||||
}
|
||||
}
|
||||
public Builder setClusterId(String value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000001;
|
||||
clusterId_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public Builder clearClusterId() {
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
clusterId_ = getDefaultInstance().getClusterId();
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
void setClusterId(com.google.protobuf.ByteString value) {
|
||||
bitField0_ |= 0x00000001;
|
||||
clusterId_ = value;
|
||||
onChanged();
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:ClusterId)
|
||||
}
|
||||
|
||||
static {
|
||||
defaultInstance = new ClusterId(true);
|
||||
defaultInstance.initFields();
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(class_scope:ClusterId)
|
||||
}
|
||||
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_ClusterId_descriptor;
|
||||
private static
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_ClusterId_fieldAccessorTable;
|
||||
|
||||
public static com.google.protobuf.Descriptors.FileDescriptor
|
||||
getDescriptor() {
|
||||
return descriptor;
|
||||
}
|
||||
private static com.google.protobuf.Descriptors.FileDescriptor
|
||||
descriptor;
|
||||
static {
|
||||
java.lang.String[] descriptorData = {
|
||||
"\n\017ClusterId.proto\"\036\n\tClusterId\022\021\n\tcluste" +
|
||||
"rId\030\001 \002(\tBB\n*org.apache.hadoop.hbase.pro" +
|
||||
"tobuf.generatedB\017ClusterIdProtosH\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
public com.google.protobuf.ExtensionRegistry assignDescriptors(
|
||||
com.google.protobuf.Descriptors.FileDescriptor root) {
|
||||
descriptor = root;
|
||||
internal_static_ClusterId_descriptor =
|
||||
getDescriptor().getMessageTypes().get(0);
|
||||
internal_static_ClusterId_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_ClusterId_descriptor,
|
||||
new java.lang.String[] { "ClusterId", },
|
||||
org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.ClusterIdProtos.ClusterId.Builder.class);
|
||||
return null;
|
||||
}
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor
|
||||
.internalBuildGeneratedFileFrom(descriptorData,
|
||||
new com.google.protobuf.Descriptors.FileDescriptor[] {
|
||||
}, assigner);
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(outer_class_scope)
|
||||
}
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -954,422 +954,6 @@ public final class ZooKeeperProtos {
|
|||
// @@protoc_insertion_point(class_scope:Master)
|
||||
}
|
||||
|
||||
public interface ClusterIdOrBuilder
|
||||
extends com.google.protobuf.MessageOrBuilder {
|
||||
|
||||
// required string clusterId = 1;
|
||||
boolean hasClusterId();
|
||||
String getClusterId();
|
||||
}
|
||||
public static final class ClusterId extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
implements ClusterIdOrBuilder {
|
||||
// Use ClusterId.newBuilder() to construct.
|
||||
private ClusterId(Builder builder) {
|
||||
super(builder);
|
||||
}
|
||||
private ClusterId(boolean noInit) {}
|
||||
|
||||
private static final ClusterId defaultInstance;
|
||||
public static ClusterId getDefaultInstance() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public ClusterId getDefaultInstanceForType() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterId_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterId_fieldAccessorTable;
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
// required string clusterId = 1;
|
||||
public static final int CLUSTERID_FIELD_NUMBER = 1;
|
||||
private java.lang.Object clusterId_;
|
||||
public boolean hasClusterId() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public String getClusterId() {
|
||||
java.lang.Object ref = clusterId_;
|
||||
if (ref instanceof String) {
|
||||
return (String) ref;
|
||||
} else {
|
||||
com.google.protobuf.ByteString bs =
|
||||
(com.google.protobuf.ByteString) ref;
|
||||
String s = bs.toStringUtf8();
|
||||
if (com.google.protobuf.Internal.isValidUtf8(bs)) {
|
||||
clusterId_ = s;
|
||||
}
|
||||
return s;
|
||||
}
|
||||
}
|
||||
private com.google.protobuf.ByteString getClusterIdBytes() {
|
||||
java.lang.Object ref = clusterId_;
|
||||
if (ref instanceof String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
|
||||
clusterId_ = b;
|
||||
return b;
|
||||
} else {
|
||||
return (com.google.protobuf.ByteString) ref;
|
||||
}
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
clusterId_ = "";
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
byte isInitialized = memoizedIsInitialized;
|
||||
if (isInitialized != -1) return isInitialized == 1;
|
||||
|
||||
if (!hasClusterId()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
memoizedIsInitialized = 1;
|
||||
return true;
|
||||
}
|
||||
|
||||
public void writeTo(com.google.protobuf.CodedOutputStream output)
|
||||
throws java.io.IOException {
|
||||
getSerializedSize();
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
output.writeBytes(1, getClusterIdBytes());
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
private int memoizedSerializedSize = -1;
|
||||
public int getSerializedSize() {
|
||||
int size = memoizedSerializedSize;
|
||||
if (size != -1) return size;
|
||||
|
||||
size = 0;
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBytesSize(1, getClusterIdBytes());
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
}
|
||||
|
||||
private static final long serialVersionUID = 0L;
|
||||
@java.lang.Override
|
||||
protected java.lang.Object writeReplace()
|
||||
throws java.io.ObjectStreamException {
|
||||
return super.writeReplace();
|
||||
}
|
||||
|
||||
@java.lang.Override
|
||||
public boolean equals(final java.lang.Object obj) {
|
||||
if (obj == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId)) {
|
||||
return super.equals(obj);
|
||||
}
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId) obj;
|
||||
|
||||
boolean result = true;
|
||||
result = result && (hasClusterId() == other.hasClusterId());
|
||||
if (hasClusterId()) {
|
||||
result = result && getClusterId()
|
||||
.equals(other.getClusterId());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
}
|
||||
|
||||
@java.lang.Override
|
||||
public int hashCode() {
|
||||
int hash = 41;
|
||||
hash = (19 * hash) + getDescriptorForType().hashCode();
|
||||
if (hasClusterId()) {
|
||||
hash = (37 * hash) + CLUSTERID_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getClusterId().hashCode();
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
return hash;
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(
|
||||
com.google.protobuf.ByteString data)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(
|
||||
com.google.protobuf.ByteString data,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(byte[] data)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(
|
||||
byte[] data,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(java.io.InputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(
|
||||
java.io.InputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseDelimitedFrom(java.io.InputStream input)
|
||||
throws java.io.IOException {
|
||||
Builder builder = newBuilder();
|
||||
if (builder.mergeDelimitedFrom(input)) {
|
||||
return builder.buildParsed();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseDelimitedFrom(
|
||||
java.io.InputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
Builder builder = newBuilder();
|
||||
if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
|
||||
return builder.buildParsed();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(
|
||||
com.google.protobuf.CodedInputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(
|
||||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
|
||||
public static Builder newBuilder() { return Builder.create(); }
|
||||
public Builder newBuilderForType() { return newBuilder(); }
|
||||
public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId prototype) {
|
||||
return newBuilder().mergeFrom(prototype);
|
||||
}
|
||||
public Builder toBuilder() { return newBuilder(this); }
|
||||
|
||||
@java.lang.Override
|
||||
protected Builder newBuilderForType(
|
||||
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
||||
Builder builder = new Builder(parent);
|
||||
return builder;
|
||||
}
|
||||
public static final class Builder extends
|
||||
com.google.protobuf.GeneratedMessage.Builder<Builder>
|
||||
implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterIdOrBuilder {
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterId_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterId_fieldAccessorTable;
|
||||
}
|
||||
|
||||
// Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.newBuilder()
|
||||
private Builder() {
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
|
||||
private Builder(BuilderParent parent) {
|
||||
super(parent);
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
private void maybeForceBuilderInitialization() {
|
||||
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
|
||||
}
|
||||
}
|
||||
private static Builder create() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public Builder clear() {
|
||||
super.clear();
|
||||
clusterId_ = "";
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder clone() {
|
||||
return create().mergeFrom(buildPartial());
|
||||
}
|
||||
|
||||
public com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptorForType() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.getDescriptor();
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId getDefaultInstanceForType() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.getDefaultInstance();
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId build() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId buildParsed()
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(
|
||||
result).asInvalidProtocolBufferException();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId buildPartial() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId(this);
|
||||
int from_bitField0_ = bitField0_;
|
||||
int to_bitField0_ = 0;
|
||||
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
to_bitField0_ |= 0x00000001;
|
||||
}
|
||||
result.clusterId_ = clusterId_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
}
|
||||
|
||||
public Builder mergeFrom(com.google.protobuf.Message other) {
|
||||
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId) {
|
||||
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId)other);
|
||||
} else {
|
||||
super.mergeFrom(other);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId other) {
|
||||
if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.getDefaultInstance()) return this;
|
||||
if (other.hasClusterId()) {
|
||||
setClusterId(other.getClusterId());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
||||
public final boolean isInitialized() {
|
||||
if (!hasClusterId()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public Builder mergeFrom(
|
||||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
|
||||
com.google.protobuf.UnknownFieldSet.newBuilder(
|
||||
this.getUnknownFields());
|
||||
while (true) {
|
||||
int tag = input.readTag();
|
||||
switch (tag) {
|
||||
case 0:
|
||||
this.setUnknownFields(unknownFields.build());
|
||||
onChanged();
|
||||
return this;
|
||||
default: {
|
||||
if (!parseUnknownField(input, unknownFields,
|
||||
extensionRegistry, tag)) {
|
||||
this.setUnknownFields(unknownFields.build());
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
break;
|
||||
}
|
||||
case 10: {
|
||||
bitField0_ |= 0x00000001;
|
||||
clusterId_ = input.readBytes();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
|
||||
// required string clusterId = 1;
|
||||
private java.lang.Object clusterId_ = "";
|
||||
public boolean hasClusterId() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public String getClusterId() {
|
||||
java.lang.Object ref = clusterId_;
|
||||
if (!(ref instanceof String)) {
|
||||
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
|
||||
clusterId_ = s;
|
||||
return s;
|
||||
} else {
|
||||
return (String) ref;
|
||||
}
|
||||
}
|
||||
public Builder setClusterId(String value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000001;
|
||||
clusterId_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public Builder clearClusterId() {
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
clusterId_ = getDefaultInstance().getClusterId();
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
void setClusterId(com.google.protobuf.ByteString value) {
|
||||
bitField0_ |= 0x00000001;
|
||||
clusterId_ = value;
|
||||
onChanged();
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:ClusterId)
|
||||
}
|
||||
|
||||
static {
|
||||
defaultInstance = new ClusterId(true);
|
||||
defaultInstance.initFields();
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(class_scope:ClusterId)
|
||||
}
|
||||
|
||||
public interface ClusterUpOrBuilder
|
||||
extends com.google.protobuf.MessageOrBuilder {
|
||||
|
||||
|
@ -3656,11 +3240,6 @@ public final class ZooKeeperProtos {
|
|||
private static
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_Master_fieldAccessorTable;
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_ClusterId_descriptor;
|
||||
private static
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_ClusterId_fieldAccessorTable;
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_ClusterUp_descriptor;
|
||||
private static
|
||||
|
@ -3693,20 +3272,19 @@ public final class ZooKeeperProtos {
|
|||
"\n\017ZooKeeper.proto\032\013hbase.proto\"/\n\020RootRe" +
|
||||
"gionServer\022\033\n\006server\030\001 \002(\0132\013.ServerName\"" +
|
||||
"%\n\006Master\022\033\n\006master\030\001 \002(\0132\013.ServerName\"\036" +
|
||||
"\n\tClusterId\022\021\n\tclusterId\030\001 \002(\t\"\036\n\tCluste" +
|
||||
"rUp\022\021\n\tstartDate\030\001 \002(\t\"\211\001\n\020RegionTransit" +
|
||||
"ion\022\025\n\reventTypeCode\030\001 \002(\r\022\022\n\nregionName" +
|
||||
"\030\002 \002(\014\022\022\n\ncreateTime\030\003 \002(\004\022%\n\020originServ" +
|
||||
"erName\030\004 \001(\0132\013.ServerName\022\017\n\007payload\030\005 \001" +
|
||||
"(\014\"\230\001\n\014SplitLogTask\022\"\n\005state\030\001 \002(\0162\023.Spl" +
|
||||
"itLogTask.State\022\037\n\nserverName\030\002 \002(\0132\013.Se",
|
||||
"rverName\"C\n\005State\022\016\n\nUNASSIGNED\020\000\022\t\n\005OWN" +
|
||||
"ED\020\001\022\014\n\010RESIGNED\020\002\022\010\n\004DONE\020\003\022\007\n\003ERR\020\004\"n\n" +
|
||||
"\005Table\022$\n\005state\030\001 \002(\0162\014.Table.State:\007ENA" +
|
||||
"BLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001" +
|
||||
"\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003BE\n*org.apa" +
|
||||
"che.hadoop.hbase.protobuf.generatedB\017Zoo" +
|
||||
"KeeperProtosH\001\210\001\001\240\001\001"
|
||||
"\n\tClusterUp\022\021\n\tstartDate\030\001 \002(\t\"\211\001\n\020Regio" +
|
||||
"nTransition\022\025\n\reventTypeCode\030\001 \002(\r\022\022\n\nre" +
|
||||
"gionName\030\002 \002(\014\022\022\n\ncreateTime\030\003 \002(\004\022%\n\020or" +
|
||||
"iginServerName\030\004 \001(\0132\013.ServerName\022\017\n\007pay" +
|
||||
"load\030\005 \001(\014\"\230\001\n\014SplitLogTask\022\"\n\005state\030\001 \002" +
|
||||
"(\0162\023.SplitLogTask.State\022\037\n\nserverName\030\002 " +
|
||||
"\002(\0132\013.ServerName\"C\n\005State\022\016\n\nUNASSIGNED\020",
|
||||
"\000\022\t\n\005OWNED\020\001\022\014\n\010RESIGNED\020\002\022\010\n\004DONE\020\003\022\007\n\003" +
|
||||
"ERR\020\004\"n\n\005Table\022$\n\005state\030\001 \002(\0162\014.Table.St" +
|
||||
"ate:\007ENABLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DI" +
|
||||
"SABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003BE\n" +
|
||||
"*org.apache.hadoop.hbase.protobuf.genera" +
|
||||
"tedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -3729,16 +3307,8 @@ public final class ZooKeeperProtos {
|
|||
new java.lang.String[] { "Master", },
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.Builder.class);
|
||||
internal_static_ClusterId_descriptor =
|
||||
getDescriptor().getMessageTypes().get(2);
|
||||
internal_static_ClusterId_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_ClusterId_descriptor,
|
||||
new java.lang.String[] { "ClusterId", },
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.Builder.class);
|
||||
internal_static_ClusterUp_descriptor =
|
||||
getDescriptor().getMessageTypes().get(3);
|
||||
getDescriptor().getMessageTypes().get(2);
|
||||
internal_static_ClusterUp_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_ClusterUp_descriptor,
|
||||
|
@ -3746,7 +3316,7 @@ public final class ZooKeeperProtos {
|
|||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.Builder.class);
|
||||
internal_static_RegionTransition_descriptor =
|
||||
getDescriptor().getMessageTypes().get(4);
|
||||
getDescriptor().getMessageTypes().get(3);
|
||||
internal_static_RegionTransition_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_RegionTransition_descriptor,
|
||||
|
@ -3754,7 +3324,7 @@ public final class ZooKeeperProtos {
|
|||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.Builder.class);
|
||||
internal_static_SplitLogTask_descriptor =
|
||||
getDescriptor().getMessageTypes().get(5);
|
||||
getDescriptor().getMessageTypes().get(4);
|
||||
internal_static_SplitLogTask_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_SplitLogTask_descriptor,
|
||||
|
@ -3762,7 +3332,7 @@ public final class ZooKeeperProtos {
|
|||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.Builder.class);
|
||||
internal_static_Table_descriptor =
|
||||
getDescriptor().getMessageTypes().get(6);
|
||||
getDescriptor().getMessageTypes().get(5);
|
||||
internal_static_Table_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_Table_descriptor,
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
|
@ -65,6 +66,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -493,10 +495,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
*/
|
||||
public long initialize(final CancelableProgressable reporter)
|
||||
throws IOException {
|
||||
|
||||
MonitoredTask status = TaskMonitor.get().createStatus(
|
||||
"Initializing region " + this);
|
||||
|
||||
MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
|
||||
long nextSeqId = -1;
|
||||
try {
|
||||
nextSeqId = initializeRegionInternals(reporter, status);
|
||||
|
@ -739,8 +738,16 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
*/
|
||||
private void checkRegioninfoOnFilesystem() throws IOException {
|
||||
Path regioninfoPath = new Path(this.regiondir, REGIONINFO_FILE);
|
||||
if (this.fs.exists(regioninfoPath) &&
|
||||
this.fs.getFileStatus(regioninfoPath).getLen() > 0) {
|
||||
// Compose the content of the file so we can compare to length in filesystem. If not same,
|
||||
// rewrite it (it may have been written in the old format using Writables instead of pb). The
|
||||
// pb version is much shorter -- we write now w/o the toString version -- so checking length
|
||||
// only should be sufficient. I don't want to read the file every time to check if it pb
|
||||
// serialized.
|
||||
byte [] content = getDotRegionInfoFileContent(this.getRegionInfo());
|
||||
boolean exists = this.fs.exists(regioninfoPath);
|
||||
FileStatus status = exists? this.fs.getFileStatus(regioninfoPath): null;
|
||||
if (status != null && status.getLen() == content.length) {
|
||||
// Then assume the content good and move on.
|
||||
return;
|
||||
}
|
||||
// Create in tmpdir and then move into place in case we crash after
|
||||
|
@ -748,14 +755,13 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// subsequent region reopens will fail the below because create is
|
||||
// registered in NN.
|
||||
|
||||
// first check to get the permissions
|
||||
FsPermission perms = FSUtils.getFilePermissions(fs, conf,
|
||||
HConstants.DATA_FILE_UMASK_KEY);
|
||||
// First check to get the permissions
|
||||
FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
|
||||
|
||||
// and then create the file
|
||||
// And then create the file
|
||||
Path tmpPath = new Path(getTmpDir(), REGIONINFO_FILE);
|
||||
|
||||
// if datanode crashes or if the RS goes down just before the close is called while trying to
|
||||
// If datanode crashes or if the RS goes down just before the close is called while trying to
|
||||
// close the created regioninfo file in the .tmp directory then on next
|
||||
// creation we will be getting AlreadyCreatedException.
|
||||
// Hence delete and create the file if exists.
|
||||
|
@ -764,18 +770,50 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
|
||||
FSDataOutputStream out = FSUtils.create(fs, tmpPath, perms);
|
||||
|
||||
try {
|
||||
this.regionInfo.write(out);
|
||||
out.write('\n');
|
||||
out.write('\n');
|
||||
out.write(Bytes.toBytes(this.regionInfo.toString()));
|
||||
// We used to write out this file as serialized Writable followed by '\n\n' and then the
|
||||
// toString of the HRegionInfo but now we just write out the pb serialized bytes so we can
|
||||
// for sure tell whether the content has been pb'd or not just by looking at file length; the
|
||||
// pb version will be shorter.
|
||||
out.write(content);
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
if (exists) {
|
||||
LOG.info("Rewriting .regioninfo file at " + regioninfoPath);
|
||||
if (!fs.delete(regioninfoPath, false)) {
|
||||
throw new IOException("Unable to remove existing " + regioninfoPath);
|
||||
}
|
||||
}
|
||||
if (!fs.rename(tmpPath, regioninfoPath)) {
|
||||
throw new IOException("Unable to rename " + tmpPath + " to " +
|
||||
regioninfoPath);
|
||||
throw new IOException("Unable to rename " + tmpPath + " to " + regioninfoPath);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param hri
|
||||
* @return Content of the file we write out to the filesystem under a region
|
||||
* @throws IOException
|
||||
*/
|
||||
private static byte [] getDotRegionInfoFileContent(final HRegionInfo hri) throws IOException {
|
||||
return hri.toDelimitedByteArray();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param fs
|
||||
* @param dir
|
||||
* @return An HRegionInfo instance gotten from the <code>.regioninfo</code> file under region dir
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HRegionInfo loadDotRegionInfoFileContent(final FileSystem fs, final Path dir)
|
||||
throws IOException {
|
||||
Path regioninfo = new Path(dir, HRegion.REGIONINFO_FILE);
|
||||
if (!fs.exists(regioninfo)) throw new FileNotFoundException(regioninfo.toString());
|
||||
FSDataInputStream in = fs.open(regioninfo);
|
||||
try {
|
||||
return HRegionInfo.parseFrom(in);
|
||||
} finally {
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -3789,10 +3827,8 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Opening region: " + info);
|
||||
}
|
||||
Path dir = HTableDescriptor.getTableDir(tableDir,
|
||||
info.getTableName());
|
||||
HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
|
||||
htd, rsServices);
|
||||
Path dir = HTableDescriptor.getTableDir(tableDir, info.getTableName());
|
||||
HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info, htd, rsServices);
|
||||
return r.openHRegion(reporter);
|
||||
}
|
||||
|
||||
|
|
|
@ -3287,7 +3287,7 @@ public class HRegionServer implements ClientProtocol,
|
|||
HRegion region = getRegion(request.getRegion());
|
||||
HRegionInfo info = region.getRegionInfo();
|
||||
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
|
||||
builder.setRegionInfo(ProtobufUtil.toRegionInfo(info));
|
||||
builder.setRegionInfo(HRegionInfo.convert(info));
|
||||
return builder.build();
|
||||
} catch (IOException ie) {
|
||||
throw new ServiceException(ie);
|
||||
|
@ -3350,8 +3350,8 @@ public class HRegionServer implements ClientProtocol,
|
|||
*/
|
||||
@Override
|
||||
@QosPriority(priority=HIGH_QOS)
|
||||
public OpenRegionResponse openRegion(final RpcController controller,
|
||||
final OpenRegionRequest request) throws ServiceException {
|
||||
public OpenRegionResponse openRegion(final RpcController controller, final OpenRegionRequest request)
|
||||
throws ServiceException {
|
||||
int versionOfOfflineNode = -1;
|
||||
if (request.hasVersionOfOfflineNode()) {
|
||||
versionOfOfflineNode = request.getVersionOfOfflineNode();
|
||||
|
@ -3359,13 +3359,11 @@ public class HRegionServer implements ClientProtocol,
|
|||
try {
|
||||
checkOpen();
|
||||
requestCount.incrementAndGet();
|
||||
OpenRegionResponse.Builder
|
||||
builder = OpenRegionResponse.newBuilder();
|
||||
OpenRegionResponse.Builder builder = OpenRegionResponse.newBuilder();
|
||||
Map<String, HTableDescriptor> htds =
|
||||
new HashMap<String, HTableDescriptor>(request.getRegionList().size());
|
||||
|
||||
for (RegionInfo regionInfo: request.getRegionList()) {
|
||||
HRegionInfo region = ProtobufUtil.toRegionInfo(regionInfo);
|
||||
HRegionInfo region = HRegionInfo.convert(regionInfo);
|
||||
checkIfRegionInTransition(region, OPEN);
|
||||
|
||||
HRegion onlineRegion = getFromOnlineRegions(region.getEncodedName());
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.Server;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.catalog.MetaEditor;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.io.Reference.Range;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -640,10 +639,10 @@ public class SplitTransaction {
|
|||
byte [] family = sf.getFamily();
|
||||
String encoded = this.hri_a.getEncodedName();
|
||||
Path storedir = Store.getStoreHomedir(splitdir, encoded, family);
|
||||
StoreFile.split(fs, storedir, sf, this.splitrow, Range.bottom);
|
||||
StoreFile.split(fs, storedir, sf, this.splitrow, false);
|
||||
encoded = this.hri_b.getEncodedName();
|
||||
storedir = Store.getStoreHomedir(splitdir, encoded, family);
|
||||
StoreFile.split(fs, storedir, sf, this.splitrow, Range.top);
|
||||
StoreFile.split(fs, storedir, sf, this.splitrow, true);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -876,7 +876,7 @@ public class StoreFile extends SchemaConfigured {
|
|||
* <code>SOME_DIRECTORY/REGIONNAME/FAMILY</code>.
|
||||
* @param f File to split.
|
||||
* @param splitRow
|
||||
* @param range
|
||||
* @param top True if we are referring to the top half of the hfile.
|
||||
* @return Path to created reference.
|
||||
* @throws IOException
|
||||
*/
|
||||
|
@ -884,10 +884,11 @@ public class StoreFile extends SchemaConfigured {
|
|||
final Path splitDir,
|
||||
final StoreFile f,
|
||||
final byte [] splitRow,
|
||||
final Reference.Range range)
|
||||
final boolean top)
|
||||
throws IOException {
|
||||
// A reference to the bottom half of the hsf store file.
|
||||
Reference r = new Reference(splitRow, range);
|
||||
Reference r =
|
||||
top? Reference.createTopReference(splitRow): Reference.createBottomReference(splitRow);
|
||||
// Add the referred-to regions name as a dot separated suffix.
|
||||
// See REF_NAME_PARSER regex above. The referred-to regions name is
|
||||
// up in the path of the passed in <code>f</code> -- parentdir is family,
|
||||
|
|
|
@ -58,7 +58,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
|||
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.zookeeper.ClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -188,7 +188,7 @@ public class ReplicationSource extends Thread
|
|||
this.metrics = new ReplicationSourceMetrics(peerClusterZnode);
|
||||
|
||||
try {
|
||||
this.clusterId = UUID.fromString(ClusterId.readClusterIdZNode(zkHelper
|
||||
this.clusterId = UUID.fromString(ZKClusterId.readClusterIdZNode(zkHelper
|
||||
.getZookeeperWatcher()));
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Could not read cluster id", ke);
|
||||
|
@ -250,7 +250,7 @@ public class ReplicationSource extends Thread
|
|||
}
|
||||
// delay this until we are in an asynchronous thread
|
||||
try {
|
||||
this.peerClusterId = UUID.fromString(ClusterId
|
||||
this.peerClusterId = UUID.fromString(ZKClusterId
|
||||
.readClusterIdZNode(zkHelper.getPeerClusters().get(peerId).getZkw()));
|
||||
} catch (KeeperException ke) {
|
||||
this.terminate("Could not read peer's cluster id", ke);
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKLeaderManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -67,7 +67,7 @@ public class AuthenticationTokenSecretManager
|
|||
private long tokenMaxLifetime;
|
||||
private ZKSecretWatcher zkWatcher;
|
||||
private LeaderElector leaderElector;
|
||||
private ClusterId clusterId;
|
||||
private ZKClusterId clusterId;
|
||||
|
||||
private Map<Integer,AuthenticationKey> allKeys =
|
||||
new ConcurrentHashMap<Integer, AuthenticationKey>();
|
||||
|
@ -96,7 +96,7 @@ public class AuthenticationTokenSecretManager
|
|||
this.tokenMaxLifetime = tokenMaxLifetime;
|
||||
this.leaderElector = new LeaderElector(zk, serverName);
|
||||
this.name = NAME_PREFIX+serverName;
|
||||
this.clusterId = new ClusterId(zk, zk);
|
||||
this.clusterId = new ZKClusterId(zk, zk);
|
||||
}
|
||||
|
||||
public void start() {
|
||||
|
|
|
@ -33,7 +33,6 @@ import java.security.PrivilegedAction;
|
|||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
|
|
@ -39,10 +39,13 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
||||
import com.google.common.primitives.Ints;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -395,15 +398,25 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
if (tableDir == null) throw new NullPointerException();
|
||||
FileStatus status = getTableInfoPath(fs, tableDir);
|
||||
if (status == null) return null;
|
||||
int len = Ints.checkedCast(status.getLen());
|
||||
byte [] content = new byte[len];
|
||||
FSDataInputStream fsDataInputStream = fs.open(status.getPath());
|
||||
HTableDescriptor hTableDescriptor = null;
|
||||
try {
|
||||
hTableDescriptor = new HTableDescriptor();
|
||||
hTableDescriptor.readFields(fsDataInputStream);
|
||||
fsDataInputStream.readFully(content);
|
||||
} finally {
|
||||
fsDataInputStream.close();
|
||||
}
|
||||
return hTableDescriptor;
|
||||
HTableDescriptor htd = null;
|
||||
try {
|
||||
htd = HTableDescriptor.parseFrom(content);
|
||||
} catch (DeserializationException e) {
|
||||
throw new IOException("content=" + Bytes.toShort(content), e);
|
||||
}
|
||||
if (!ProtobufUtil.isPBMagicPrefix(content)) {
|
||||
// Convert the file over to be pb before leaving here.
|
||||
createTableDescriptor(fs, tableDir.getParent(), htd, true);
|
||||
}
|
||||
return htd;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -451,16 +464,14 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
final HTableDescriptor hTableDescriptor, final Path tableDir,
|
||||
final FileStatus status)
|
||||
throws IOException {
|
||||
// Get temporary dir into which we'll first write a file to avoid
|
||||
// half-written file phenomeon.
|
||||
// Get temporary dir into which we'll first write a file to avoid half-written file phenomenon.
|
||||
Path tmpTableDir = new Path(tableDir, ".tmp");
|
||||
// What is current sequenceid? We read the current sequenceid from
|
||||
// the current file. After we read it, another thread could come in and
|
||||
// compete with us writing out next version of file. The below retries
|
||||
// should help in this case some but its hard to do guarantees in face of
|
||||
// concurrent schema edits.
|
||||
int currentSequenceid =
|
||||
status == null? 0: getTableInfoSequenceid(status.getPath());
|
||||
int currentSequenceid = status == null? 0: getTableInfoSequenceid(status.getPath());
|
||||
int sequenceid = currentSequenceid;
|
||||
// Put arbitrary upperbound on how often we retry
|
||||
int retries = 10;
|
||||
|
@ -499,15 +510,13 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
return tableInfoPath;
|
||||
}
|
||||
|
||||
private static void writeHTD(final FileSystem fs, final Path p,
|
||||
final HTableDescriptor htd)
|
||||
private static void writeHTD(final FileSystem fs, final Path p, final HTableDescriptor htd)
|
||||
throws IOException {
|
||||
FSDataOutputStream out = fs.create(p, false);
|
||||
try {
|
||||
htd.write(out);
|
||||
out.write('\n');
|
||||
out.write('\n');
|
||||
out.write(Bytes.toBytes(htd.toString()));
|
||||
// We used to write this file out as a serialized HTD Writable followed by two '\n's and then
|
||||
// the toString version of HTD. Now we just write out the pb serialization.
|
||||
out.write(htd.toByteArray());
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
|
@ -538,8 +547,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
final Configuration conf, boolean forceCreation)
|
||||
throws IOException {
|
||||
FileSystem fs = FSUtils.getCurrentFileSystem(conf);
|
||||
return createTableDescriptor(fs, FSUtils.getRootDir(conf), htableDescriptor,
|
||||
forceCreation);
|
||||
return createTableDescriptor(fs, FSUtils.getRootDir(conf), htableDescriptor, forceCreation);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -569,8 +577,7 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
public static boolean createTableDescriptor(FileSystem fs, Path rootdir,
|
||||
HTableDescriptor htableDescriptor, boolean forceCreation)
|
||||
throws IOException {
|
||||
FileStatus status =
|
||||
getTableInfoPath(fs, rootdir, htableDescriptor.getNameAsString());
|
||||
FileStatus status = getTableInfoPath(fs, rootdir, htableDescriptor.getNameAsString());
|
||||
if (status != null) {
|
||||
LOG.info("Current tableInfoPath = " + status.getPath());
|
||||
if (!forceCreation) {
|
||||
|
|
|
@ -19,10 +19,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.EOFException;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -43,17 +45,25 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.ClusterId;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* Utility methods for interacting with the underlying file system.
|
||||
*/
|
||||
|
@ -252,25 +262,74 @@ public abstract class FSUtils {
|
|||
* @param rootdir root hbase directory
|
||||
* @return null if no version file exists, version string otherwise.
|
||||
* @throws IOException e
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
public static String getVersion(FileSystem fs, Path rootdir)
|
||||
throws IOException {
|
||||
throws IOException, DeserializationException {
|
||||
Path versionFile = new Path(rootdir, HConstants.VERSION_FILE_NAME);
|
||||
FileStatus [] status = fs.listStatus(versionFile);
|
||||
if (status == null || status.length == 0) return null;
|
||||
String version = null;
|
||||
if (fs.exists(versionFile)) {
|
||||
FSDataInputStream s =
|
||||
fs.open(versionFile);
|
||||
byte [] content = new byte [(int)status[0].getLen()];
|
||||
FSDataInputStream s = fs.open(versionFile);
|
||||
try {
|
||||
version = DataInputStream.readUTF(s);
|
||||
IOUtils.readFully(s, content, 0, content.length);
|
||||
if (ProtobufUtil.isPBMagicPrefix(content)) {
|
||||
version = parseVersionFrom(content);
|
||||
} else {
|
||||
// Presume it pre-pb format.
|
||||
InputStream is = new ByteArrayInputStream(content);
|
||||
DataInputStream dis = new DataInputStream(is);
|
||||
try {
|
||||
version = dis.readUTF();
|
||||
} finally {
|
||||
dis.close();
|
||||
}
|
||||
// Update the format
|
||||
LOG.info("Updating the hbase.version file format with version=" + version);
|
||||
setVersion(fs, rootdir, version, 0, HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS);
|
||||
}
|
||||
} catch (EOFException eof) {
|
||||
LOG.warn("Version file was empty, odd, will try to set it.");
|
||||
} finally {
|
||||
s.close();
|
||||
}
|
||||
}
|
||||
return version;
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse the content of the ${HBASE_ROOTDIR}/hbase.version file.
|
||||
* @param bytes The byte content of the hbase.version file.
|
||||
* @return The version found in the file as a String.
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
static String parseVersionFrom(final byte [] bytes)
|
||||
throws DeserializationException {
|
||||
ProtobufUtil.expectPBMagicPrefix(bytes);
|
||||
int pblen = ProtobufUtil.lengthOfPBMagic();
|
||||
FSProtos.HBaseVersionFileContent.Builder builder =
|
||||
FSProtos.HBaseVersionFileContent.newBuilder();
|
||||
FSProtos.HBaseVersionFileContent fileContent;
|
||||
try {
|
||||
fileContent = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
|
||||
return fileContent.getVersion();
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
// Convert
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the content to write into the ${HBASE_ROOTDIR}/hbase.version file.
|
||||
* @param version Version to persist
|
||||
* @return Serialized protobuf with <code>version</code> content and a bit of pb magic for a prefix.
|
||||
*/
|
||||
static byte [] toVersionByteArray(final String version) {
|
||||
FSProtos.HBaseVersionFileContent.Builder builder =
|
||||
FSProtos.HBaseVersionFileContent.newBuilder();
|
||||
return ProtobufUtil.prependPBMagic(builder.setVersion(version).build().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies current version of file system
|
||||
*
|
||||
|
@ -279,11 +338,11 @@ public abstract class FSUtils {
|
|||
* @param message if true, issues a message on System.out
|
||||
*
|
||||
* @throws IOException e
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
public static void checkVersion(FileSystem fs, Path rootdir,
|
||||
boolean message) throws IOException {
|
||||
checkVersion(fs, rootdir, message, 0,
|
||||
HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS);
|
||||
public static void checkVersion(FileSystem fs, Path rootdir, boolean message)
|
||||
throws IOException, DeserializationException {
|
||||
checkVersion(fs, rootdir, message, 0, HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -296,20 +355,20 @@ public abstract class FSUtils {
|
|||
* @param retries number of times to retry
|
||||
*
|
||||
* @throws IOException e
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
public static void checkVersion(FileSystem fs, Path rootdir,
|
||||
boolean message, int wait, int retries) throws IOException {
|
||||
boolean message, int wait, int retries)
|
||||
throws IOException, DeserializationException {
|
||||
String version = getVersion(fs, rootdir);
|
||||
|
||||
if (version == null) {
|
||||
if (!rootRegionExists(fs, rootdir)) {
|
||||
// rootDir is empty (no version file and no root region)
|
||||
// just create new version file (HBASE-1195)
|
||||
FSUtils.setVersion(fs, rootdir, wait, retries);
|
||||
setVersion(fs, rootdir, wait, retries);
|
||||
return;
|
||||
}
|
||||
} else if (version.compareTo(HConstants.FILE_SYSTEM_VERSION) == 0)
|
||||
return;
|
||||
} else if (version.compareTo(HConstants.FILE_SYSTEM_VERSION) == 0) return;
|
||||
|
||||
// version is deprecated require migration
|
||||
// Output on stdout so user sees it in terminal.
|
||||
|
@ -367,15 +426,13 @@ public abstract class FSUtils {
|
|||
while (true) {
|
||||
try {
|
||||
FSDataOutputStream s = fs.create(versionFile);
|
||||
s.writeUTF(version);
|
||||
LOG.debug("Created version file at " + rootdir.toString() +
|
||||
" set its version at:" + version);
|
||||
s.write(toVersionByteArray(version));
|
||||
s.close();
|
||||
LOG.debug("Created version file at " + rootdir.toString() + " with version=" + version);
|
||||
return;
|
||||
} catch (IOException e) {
|
||||
if (retries > 0) {
|
||||
LOG.warn("Unable to create version file at " + rootdir.toString() +
|
||||
", retrying: " + e.getMessage());
|
||||
LOG.warn("Unable to create version file at " + rootdir.toString() + ", retrying", e);
|
||||
fs.delete(versionFile, false);
|
||||
try {
|
||||
if (wait > 0) {
|
||||
|
@ -431,25 +488,54 @@ public abstract class FSUtils {
|
|||
* @return the unique cluster identifier
|
||||
* @throws IOException if reading the cluster ID file fails
|
||||
*/
|
||||
public static String getClusterId(FileSystem fs, Path rootdir)
|
||||
public static ClusterId getClusterId(FileSystem fs, Path rootdir)
|
||||
throws IOException {
|
||||
Path idPath = new Path(rootdir, HConstants.CLUSTER_ID_FILE_NAME);
|
||||
String clusterId = null;
|
||||
if (fs.exists(idPath)) {
|
||||
ClusterId clusterId = null;
|
||||
FileStatus status = fs.exists(idPath)? fs.getFileStatus(idPath): null;
|
||||
if (status != null) {
|
||||
int len = Ints.checkedCast(status.getLen());
|
||||
byte [] content = new byte[len];
|
||||
FSDataInputStream in = fs.open(idPath);
|
||||
try {
|
||||
clusterId = in.readUTF();
|
||||
in.readFully(content);
|
||||
} catch (EOFException eof) {
|
||||
LOG.warn("Cluster ID file " + idPath.toString() + " was empty");
|
||||
} finally{
|
||||
in.close();
|
||||
}
|
||||
try {
|
||||
clusterId = ClusterId.parseFrom(content);
|
||||
} catch (DeserializationException e) {
|
||||
throw new IOException("content=" + Bytes.toString(content), e);
|
||||
}
|
||||
// If not pb'd, make it so.
|
||||
if (!ProtobufUtil.isPBMagicPrefix(content)) rewriteAsPb(fs, rootdir, idPath, clusterId);
|
||||
return clusterId;
|
||||
} else {
|
||||
LOG.warn("Cluster ID file does not exist at " + idPath.toString());
|
||||
}
|
||||
return clusterId;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param cid
|
||||
* @throws IOException
|
||||
*/
|
||||
private static void rewriteAsPb(final FileSystem fs, final Path rootdir, final Path p,
|
||||
final ClusterId cid)
|
||||
throws IOException {
|
||||
// Rewrite the file as pb. Move aside the old one first, write new
|
||||
// then delete the moved-aside file.
|
||||
Path movedAsideName = new Path(p + "." + System.currentTimeMillis());
|
||||
if (!fs.rename(p, movedAsideName)) throw new IOException("Failed rename of " + p);
|
||||
setClusterId(fs, rootdir, cid, 100);
|
||||
if (!fs.delete(movedAsideName, false)) {
|
||||
throw new IOException("Failed delete of " + movedAsideName);
|
||||
}
|
||||
LOG.debug("Rewrote the hbase.id file as pb");
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a new unique identifier for this cluster to the "hbase.id" file
|
||||
* in the HBase root directory
|
||||
|
@ -459,17 +545,19 @@ public abstract class FSUtils {
|
|||
* @param wait how long (in milliseconds) to wait between retries
|
||||
* @throws IOException if writing to the FileSystem fails and no wait value
|
||||
*/
|
||||
public static void setClusterId(FileSystem fs, Path rootdir, String clusterId,
|
||||
public static void setClusterId(FileSystem fs, Path rootdir, ClusterId clusterId,
|
||||
int wait) throws IOException {
|
||||
while (true) {
|
||||
try {
|
||||
Path filePath = new Path(rootdir, HConstants.CLUSTER_ID_FILE_NAME);
|
||||
FSDataOutputStream s = fs.create(filePath);
|
||||
s.writeUTF(clusterId);
|
||||
try {
|
||||
s.write(clusterId.toByteArray());
|
||||
} finally {
|
||||
s.close();
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Created cluster ID file at " + filePath.toString() +
|
||||
" with ID: " + clusterId);
|
||||
LOG.debug("Created cluster ID file at " + filePath.toString() + " with ID: " + clusterId);
|
||||
}
|
||||
return;
|
||||
} catch (IOException ioe) {
|
||||
|
@ -669,6 +757,7 @@ public abstract class FSUtils {
|
|||
*
|
||||
* @param master The master defining the HBase root and file system.
|
||||
* @return A map for each table and its percentage.
|
||||
*
|
||||
* @throws IOException When scanning the directory fails.
|
||||
*/
|
||||
public static Map<String, Integer> getTableFragmentation(
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -599,14 +598,7 @@ public class HBaseFsck {
|
|||
// already loaded data
|
||||
return;
|
||||
}
|
||||
|
||||
Path regioninfo = new Path(regionDir, HRegion.REGIONINFO_FILE);
|
||||
FileSystem fs = regioninfo.getFileSystem(conf);
|
||||
|
||||
FSDataInputStream in = fs.open(regioninfo);
|
||||
HRegionInfo hri = new HRegionInfo();
|
||||
hri.readFields(in);
|
||||
in.close();
|
||||
HRegionInfo hri = HRegion.loadDotRegionInfoFileContent(FileSystem.get(this.conf), regionDir);
|
||||
LOG.debug("HRegionInfo read: " + hri.toString());
|
||||
hbi.hdfsEntry.hri = hri;
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.util;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
|
@ -143,6 +142,7 @@ 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 {
|
||||
|
@ -182,6 +182,7 @@ public class Writables {
|
|||
* @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 {
|
||||
|
|
|
@ -22,12 +22,10 @@ package org.apache.hadoop.hbase.zookeeper;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.ClusterId;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* Publishes and synchronizes a unique identifier specific to a given HBase
|
||||
* cluster. The stored identifier is read from the file system by the active
|
||||
|
@ -35,12 +33,12 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
|||
* clients).
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ClusterId {
|
||||
public class ZKClusterId {
|
||||
private ZooKeeperWatcher watcher;
|
||||
private Abortable abortable;
|
||||
private String id;
|
||||
|
||||
public ClusterId(ZooKeeperWatcher watcher, Abortable abortable) {
|
||||
public ZKClusterId(ZooKeeperWatcher watcher, Abortable abortable) {
|
||||
this.watcher = watcher;
|
||||
this.abortable = abortable;
|
||||
}
|
||||
|
@ -66,45 +64,18 @@ public class ClusterId {
|
|||
if (ZKUtil.checkExists(watcher, watcher.clusterIdZNode) != -1) {
|
||||
byte [] data = ZKUtil.getData(watcher, watcher.clusterIdZNode);
|
||||
if (data != null) {
|
||||
return getZNodeClusterId(data);
|
||||
try {
|
||||
return ClusterId.parseFrom(data).toString();
|
||||
} catch (DeserializationException e) {
|
||||
throw ZKUtil.convert(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public static void setClusterId(ZooKeeperWatcher watcher, String id)
|
||||
public static void setClusterId(ZooKeeperWatcher watcher, ClusterId id)
|
||||
throws KeeperException {
|
||||
ZKUtil.createSetData(watcher, watcher.clusterIdZNode, getZNodeData(id));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param clusterid
|
||||
* @return Content of the clusterid znode as a serialized pb with the pb
|
||||
* magic as prefix.
|
||||
*/
|
||||
static byte [] getZNodeData(final String clusterid) {
|
||||
ZooKeeperProtos.ClusterId.Builder builder =
|
||||
ZooKeeperProtos.ClusterId.newBuilder();
|
||||
builder.setClusterId(clusterid);
|
||||
return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param data
|
||||
* @return The clusterid extracted from the passed znode <code>data</code>
|
||||
*/
|
||||
static String getZNodeClusterId(final byte [] data) {
|
||||
if (data == null || data.length <= 0) return null;
|
||||
// If no magic, something is seriously wrong. Fail fast.
|
||||
if (!ProtobufUtil.isPBMagicPrefix(data)) throw new RuntimeException("No magic preamble");
|
||||
int prefixLen = ProtobufUtil.lengthOfPBMagic();
|
||||
try {
|
||||
ZooKeeperProtos.ClusterId clusterid =
|
||||
ZooKeeperProtos.ClusterId.newBuilder().mergeFrom(data, prefixLen, data.length - prefixLen).build();
|
||||
return clusterid.getClusterId();
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
// A failed parse of the znode is pretty catastrophic. Fail fast.
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
ZKUtil.createSetData(watcher, watcher.clusterIdZNode, id.toByteArray());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
// This file contains protocol buffers that are shared throughout HBase
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_outer_classname = "ClusterIdProtos";
|
||||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
/**
|
||||
* Content of the '/hbase/hbaseid', cluster id, znode.
|
||||
* Also cluster of the ${HBASE_ROOTDIR}/hbase.id file.
|
||||
*/
|
||||
message ClusterId {
|
||||
// This is the cluster id, a uuid as a String
|
||||
required string clusterId = 1;
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
// This file contains protocol buffers that are written into the filesystem
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_outer_classname = "FSProtos";
|
||||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
/**
|
||||
* The ${HBASE_ROOTDIR}/hbase.version file content
|
||||
*/
|
||||
message HBaseVersionFileContent {
|
||||
required string version = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reference file content used when we split an hfile under a region.
|
||||
*/
|
||||
message Reference {
|
||||
required bytes splitkey = 1;
|
||||
enum Range {
|
||||
TOP = 0;
|
||||
BOTTOM = 1;
|
||||
}
|
||||
required Range range = 2;
|
||||
}
|
||||
|
|
@ -43,15 +43,6 @@ message Master {
|
|||
required ServerName master = 1;
|
||||
}
|
||||
|
||||
// TODO: Put these two cluster attributes into the one znode.
|
||||
/**
|
||||
* Content of the '/hbase/hbaseid', cluster id, znode.
|
||||
*/
|
||||
message ClusterId {
|
||||
// This is the cluster id, a uuid as a String
|
||||
required string clusterId = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Content of the '/hbase/shutdown', cluster state, znode.
|
||||
*/
|
||||
|
|
|
@ -23,6 +23,33 @@ option java_outer_classname = "HBaseProtos";
|
|||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
/**
|
||||
* Table Schema
|
||||
* Inspired by the rest TableSchema
|
||||
*/
|
||||
message TableSchema {
|
||||
optional bytes name = 1;
|
||||
message Attribute {
|
||||
required bytes name = 1;
|
||||
required bytes value = 2;
|
||||
}
|
||||
repeated Attribute attributes = 2;
|
||||
repeated ColumnFamilySchema columnFamilies = 3;
|
||||
}
|
||||
|
||||
/**
|
||||
* Column Family Schema
|
||||
* Inspired by the rest ColumSchemaMessage
|
||||
*/
|
||||
message ColumnFamilySchema {
|
||||
required bytes name = 1;
|
||||
message Attribute {
|
||||
required bytes name = 1;
|
||||
required bytes value = 2;
|
||||
}
|
||||
repeated Attribute attributes = 2;
|
||||
}
|
||||
|
||||
/**
|
||||
* Protocol buffer version of HRegionInfo.
|
||||
*/
|
||||
|
@ -196,4 +223,3 @@ message NameBytesPair {
|
|||
required string name = 1;
|
||||
optional bytes value = 2;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,12 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.hfile.Compression;
|
||||
import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.junit.Test;
|
||||
|
@ -25,9 +31,44 @@ import org.junit.Test;
|
|||
/** Tests the HColumnDescriptor with appropriate arguments */
|
||||
@Category(SmallTests.class)
|
||||
public class TestHColumnDescriptor {
|
||||
@Test
|
||||
public void testPb() throws DeserializationException {
|
||||
HColumnDescriptor hcd = HTableDescriptor.META_TABLEDESC.getColumnFamilies()[0];
|
||||
final int v = 123;
|
||||
hcd.setBlocksize(v);
|
||||
hcd.setTimeToLive(v);
|
||||
hcd.setBlockCacheEnabled(!HColumnDescriptor.DEFAULT_BLOCKCACHE);
|
||||
hcd.setValue("a", "b");
|
||||
hcd.setMaxVersions(v);
|
||||
assertEquals(v, hcd.getMaxVersions());
|
||||
hcd.setMinVersions(v);
|
||||
assertEquals(v, hcd.getMinVersions());
|
||||
hcd.setKeepDeletedCells(!HColumnDescriptor.DEFAULT_KEEP_DELETED);
|
||||
hcd.setInMemory(!HColumnDescriptor.DEFAULT_IN_MEMORY);
|
||||
boolean inmemory = hcd.isInMemory();
|
||||
hcd.setScope(v);
|
||||
hcd.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
|
||||
hcd.setBloomFilterType(StoreFile.BloomType.ROW);
|
||||
hcd.setCompressionType(Algorithm.SNAPPY);
|
||||
|
||||
|
||||
byte [] bytes = hcd.toByteArray();
|
||||
HColumnDescriptor deserializedHcd = HColumnDescriptor.parseFrom(bytes);
|
||||
assertTrue(hcd.equals(deserializedHcd));
|
||||
assertEquals(v, hcd.getBlocksize());
|
||||
assertEquals(v, hcd.getTimeToLive());
|
||||
assertEquals(hcd.getValue("a"), deserializedHcd.getValue("a"));
|
||||
assertEquals(hcd.getMaxVersions(), deserializedHcd.getMaxVersions());
|
||||
assertEquals(hcd.getMinVersions(), deserializedHcd.getMinVersions());
|
||||
assertEquals(hcd.getKeepDeletedCells(), deserializedHcd.getKeepDeletedCells());
|
||||
assertEquals(inmemory, deserializedHcd.isInMemory());
|
||||
assertEquals(hcd.getScope(), deserializedHcd.getScope());
|
||||
assertTrue(deserializedHcd.getCompressionType().equals(Compression.Algorithm.SNAPPY));
|
||||
assertTrue(deserializedHcd.getDataBlockEncoding().equals(DataBlockEncoding.FAST_DIFF));
|
||||
assertTrue(deserializedHcd.getBloomFilterType().equals(StoreFile.BloomType.ROW));
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
/** Tests HColumnDescriptor with empty familyName*/
|
||||
public void testHColumnDescriptorShouldThrowIAEWhenFamiliyNameEmpty()
|
||||
throws Exception {
|
||||
|
|
|
@ -21,6 +21,8 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -30,6 +32,20 @@ import org.junit.experimental.categories.Category;
|
|||
*/
|
||||
@Category(SmallTests.class)
|
||||
public class TestHTableDescriptor {
|
||||
@Test
|
||||
public void testPb() throws DeserializationException, IOException {
|
||||
HTableDescriptor htd = HTableDescriptor.META_TABLEDESC;
|
||||
final int v = 123;
|
||||
htd.setMaxFileSize(v);
|
||||
htd.setDeferredLogFlush(true);
|
||||
htd.setReadOnly(true);
|
||||
byte [] bytes = htd.toByteArray();
|
||||
HTableDescriptor deserializedHtd = HTableDescriptor.parseFrom(bytes);
|
||||
assertEquals(htd, deserializedHtd);
|
||||
assertEquals(v, deserializedHtd.getMaxFileSize());
|
||||
assertTrue(deserializedHtd.isReadOnly());
|
||||
assertTrue(deserializedHtd.isDeferredLogFlush());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test cps in the table description
|
||||
|
@ -62,5 +78,4 @@ public class TestHTableDescriptor {
|
|||
desc.remove(key);
|
||||
assertEquals(null, desc.getValue(key));
|
||||
}
|
||||
|
||||
}
|
|
@ -50,7 +50,6 @@ 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.protobuf.ProtobufUtil;
|
||||
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;
|
||||
|
@ -205,7 +204,7 @@ public class TestCatalogTracker {
|
|||
// If a 'getRegionInfo' is called on mocked AdminProtocol, throw IOE
|
||||
// the first time. 'Succeed' the second time we are called.
|
||||
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
|
||||
builder.setRegionInfo(ProtobufUtil.toRegionInfo(new HRegionInfo(Bytes.toBytes("test"))));
|
||||
builder.setRegionInfo(HRegionInfo.convert(new HRegionInfo(Bytes.toBytes("test"))));
|
||||
Mockito.when(admin.getRegionInfo((RpcController)Mockito.any(),
|
||||
(GetRegionInfoRequest)Mockito.any())).thenThrow(
|
||||
new ServiceException(new IOException("Server not running, aborting"))).
|
||||
|
@ -412,7 +411,7 @@ public class TestCatalogTracker {
|
|||
Mockito.when(connection.getRegionServerWithRetries((ServerCallable<Result>)Mockito.any())).
|
||||
thenReturn(result);
|
||||
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
|
||||
builder.setRegionInfo(ProtobufUtil.toRegionInfo(HRegionInfo.FIRST_META_REGIONINFO));
|
||||
builder.setRegionInfo(HRegionInfo.convert(HRegionInfo.FIRST_META_REGIONINFO));
|
||||
Mockito.when(implementation.getRegionInfo((RpcController)Mockito.any(),
|
||||
(GetRegionInfoRequest)Mockito.any())).thenReturn(builder.build());
|
||||
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
|
||||
|
|
|
@ -163,8 +163,7 @@ public class TestMultipleTimestamps {
|
|||
IOException {
|
||||
LOG.info("testReseeksWithMultipleColumnMultipleTimestamp");
|
||||
|
||||
byte [] TABLE = Bytes.toBytes("testReseeksWithMultiple" +
|
||||
"ColumnMiltipleTimestamps");
|
||||
byte [] TABLE = Bytes.toBytes("testReseeksWithMultipleColumnMiltipleTimestamps");
|
||||
byte [] FAMILY = Bytes.toBytes("event_log");
|
||||
byte [][] FAMILIES = new byte[][] { FAMILY };
|
||||
|
||||
|
@ -183,12 +182,20 @@ public class TestMultipleTimestamps {
|
|||
put(ht, FAMILY, putRows, putColumns, putTimestamps);
|
||||
|
||||
TEST_UTIL.flush(TABLE);
|
||||
|
||||
ResultScanner scanner = scan(ht, FAMILY, scanRows, scanColumns,
|
||||
scanTimestamps, scanMaxVersions);
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(10);
|
||||
ResultScanner scanner = ht.getScanner(scan);
|
||||
while (true) {
|
||||
Result r = scanner.next();
|
||||
if (r == null) break;
|
||||
LOG.info("r=" + r);
|
||||
}
|
||||
scanner = scan(ht, FAMILY, scanRows, scanColumns, scanTimestamps, scanMaxVersions);
|
||||
|
||||
KeyValue[] kvs;
|
||||
|
||||
// This looks like wrong answer. Should be 2. Even then we are returning wrong result,
|
||||
// timestamps that are 3 whereas should be 2 since min is inclusive.
|
||||
kvs = scanner.next().raw();
|
||||
assertEquals(4, kvs.length);
|
||||
checkOneCell(kvs[0], FAMILY, 5, 3, 3);
|
||||
|
@ -275,12 +282,8 @@ public class TestMultipleTimestamps {
|
|||
}
|
||||
|
||||
public void testWithVersionDeletes(boolean flushTables) throws IOException {
|
||||
LOG.info("testWithVersionDeletes_"+
|
||||
(flushTables ? "flush" : "noflush"));
|
||||
|
||||
byte [] TABLE = Bytes.toBytes("testWithVersionDeletes_" +
|
||||
(flushTables ? "flush" : "noflush"));
|
||||
|
||||
LOG.info("testWithVersionDeletes_"+ (flushTables ? "flush" : "noflush"));
|
||||
byte [] TABLE = Bytes.toBytes("testWithVersionDeletes_" + (flushTables ? "flush" : "noflush"));
|
||||
byte [] FAMILY = Bytes.toBytes("event_log");
|
||||
byte [][] FAMILIES = new byte[][] { FAMILY };
|
||||
|
||||
|
|
|
@ -422,7 +422,7 @@ class MockRegionServer implements AdminProtocol, ClientProtocol, RegionServerSer
|
|||
public GetRegionInfoResponse getRegionInfo(RpcController controller,
|
||||
GetRegionInfoRequest request) throws ServiceException {
|
||||
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
|
||||
builder.setRegionInfo(ProtobufUtil.toRegionInfo(HRegionInfo.ROOT_REGIONINFO));
|
||||
builder.setRegionInfo(HRegionInfo.convert(HRegionInfo.ROOT_REGIONINFO));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -344,7 +344,7 @@ public class TestCatalogJanitor {
|
|||
HTableDescriptor.getTableDir(rootdir, htd.getName());
|
||||
Path storedir = Store.getStoreHomedir(tabledir, splita.getEncodedName(),
|
||||
htd.getColumnFamilies()[0].getName());
|
||||
Reference ref = new Reference(Bytes.toBytes("ccc"), Reference.Range.top);
|
||||
Reference ref = Reference.createTopReference(Bytes.toBytes("ccc"));
|
||||
long now = System.currentTimeMillis();
|
||||
// Reference name has this format: StoreFile#REF_NAME_PARSER
|
||||
Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
|
||||
|
@ -518,8 +518,8 @@ public class TestCatalogJanitor {
|
|||
Path tabledir = HTableDescriptor.getTableDir(rootdir, parent.getTableName());
|
||||
Path storedir = Store.getStoreHomedir(tabledir, daughter.getEncodedName(),
|
||||
htd.getColumnFamilies()[0].getName());
|
||||
Reference ref = new Reference(midkey,
|
||||
top? Reference.Range.top: Reference.Range.bottom);
|
||||
Reference ref =
|
||||
top? Reference.createTopReference(midkey): Reference.createBottomReference(midkey);
|
||||
long now = System.currentTimeMillis();
|
||||
// Reference name has this format: StoreFile#REF_NAME_PARSER
|
||||
Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
|
||||
|
|
|
@ -27,16 +27,60 @@ import static org.junit.Assert.fail;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.MD5Hash;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestHRegionInfo {
|
||||
@Test
|
||||
public void testPb() throws DeserializationException {
|
||||
HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
|
||||
byte [] bytes = hri.toByteArray();
|
||||
HRegionInfo pbhri = HRegionInfo.parseFrom(bytes);
|
||||
assertTrue(hri.equals(pbhri));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadAndWriteHRegionInfoFile() throws IOException, InterruptedException {
|
||||
HBaseTestingUtility htu = new HBaseTestingUtility();
|
||||
HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
|
||||
Path basedir = htu.getDataTestDir();
|
||||
// Create a region. That'll write the .regioninfo file.
|
||||
HRegion r = HRegion.createHRegion(hri, basedir, htu.getConfiguration(),
|
||||
HTableDescriptor.META_TABLEDESC);
|
||||
// Get modtime on the file.
|
||||
long modtime = getModTime(r);
|
||||
HRegion.closeHRegion(r);
|
||||
Thread.sleep(1001);
|
||||
r = HRegion.createHRegion(hri, basedir, htu.getConfiguration(), HTableDescriptor.META_TABLEDESC);
|
||||
// Ensure the file is not written for a second time.
|
||||
long modtime2 = getModTime(r);
|
||||
assertEquals(modtime, modtime2);
|
||||
// Now load the file.
|
||||
HRegionInfo deserializedHri =
|
||||
HRegion.loadDotRegionInfoFileContent(FileSystem.get(htu.getConfiguration()), r.getRegionDir());
|
||||
assertTrue(hri.equals(deserializedHri));
|
||||
}
|
||||
|
||||
long getModTime(final HRegion r) throws IOException {
|
||||
FileStatus [] statuses =
|
||||
r.getFilesystem().listStatus(new Path(r.getRegionDir(), HRegion.REGIONINFO_FILE));
|
||||
assertTrue(statuses != null && statuses.length == 1);
|
||||
return statuses[0].getModificationTime();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateHRegionInfoName() throws Exception {
|
||||
String tableName = "tablename";
|
||||
|
|
|
@ -204,14 +204,10 @@ public class TestSplitTransaction {
|
|||
// to be under the daughter region dirs.
|
||||
assertEquals(0, this.fs.listStatus(st.getSplitDir()).length);
|
||||
// Check daughters have correct key span.
|
||||
assertTrue(Bytes.equals(this.parent.getStartKey(),
|
||||
daughters.getFirst().getStartKey()));
|
||||
assertTrue(Bytes.equals(GOOD_SPLIT_ROW,
|
||||
daughters.getFirst().getEndKey()));
|
||||
assertTrue(Bytes.equals(daughters.getSecond().getStartKey(),
|
||||
GOOD_SPLIT_ROW));
|
||||
assertTrue(Bytes.equals(this.parent.getEndKey(),
|
||||
daughters.getSecond().getEndKey()));
|
||||
assertTrue(Bytes.equals(this.parent.getStartKey(), daughters.getFirst().getStartKey()));
|
||||
assertTrue(Bytes.equals(GOOD_SPLIT_ROW, daughters.getFirst().getEndKey()));
|
||||
assertTrue(Bytes.equals(daughters.getSecond().getStartKey(), GOOD_SPLIT_ROW));
|
||||
assertTrue(Bytes.equals(this.parent.getEndKey(), daughters.getSecond().getEndKey()));
|
||||
// Count rows.
|
||||
int daughtersRowCount = 0;
|
||||
for (HRegion r: daughters) {
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.Reference.Range;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockCache;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
@ -156,7 +155,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
kv = KeyValue.createKeyValueFromKey(reader.getLastKey());
|
||||
byte [] finalRow = kv.getRow();
|
||||
// Make a reference
|
||||
Path refPath = StoreFile.split(fs, dir, hsf, midRow, Range.top);
|
||||
Path refPath = StoreFile.split(fs, dir, hsf, midRow, true);
|
||||
StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf,
|
||||
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
|
||||
// Now confirm that I can read from the reference and that it only gets
|
||||
|
@ -184,15 +183,14 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
if (this.fs.exists(topDir)) {
|
||||
this.fs.delete(topDir, true);
|
||||
}
|
||||
Path topPath = StoreFile.split(this.fs, topDir, f, midRow, Range.top);
|
||||
Path topPath = StoreFile.split(this.fs, topDir, f, midRow, true);
|
||||
// Create bottom split.
|
||||
Path bottomDir = Store.getStoreHomedir(this.testDir, "2",
|
||||
Bytes.toBytes(f.getPath().getParent().getName()));
|
||||
if (this.fs.exists(bottomDir)) {
|
||||
this.fs.delete(bottomDir, true);
|
||||
}
|
||||
Path bottomPath = StoreFile.split(this.fs, bottomDir,
|
||||
f, midRow, Range.bottom);
|
||||
Path bottomPath = StoreFile.split(this.fs, bottomDir, f, midRow, false);
|
||||
// Make readers on top and bottom.
|
||||
StoreFile.Reader top =
|
||||
new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE,
|
||||
|
@ -251,9 +249,8 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
// First, do a key that is < than first key. Ensure splits behave
|
||||
// properly.
|
||||
byte [] badmidkey = Bytes.toBytes(" .");
|
||||
topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
|
||||
bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
|
||||
Range.bottom);
|
||||
topPath = StoreFile.split(this.fs, topDir, f, badmidkey, true);
|
||||
bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey, false);
|
||||
top = new StoreFile(this.fs, topPath, conf, cacheConf,
|
||||
StoreFile.BloomType.NONE,
|
||||
NoOpDataBlockEncoder.INSTANCE).createReader();
|
||||
|
@ -298,9 +295,8 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
|
||||
// Test when badkey is > than last key in file ('||' > 'zz').
|
||||
badmidkey = Bytes.toBytes("|||");
|
||||
topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
|
||||
bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
|
||||
Range.bottom);
|
||||
topPath = StoreFile.split(this.fs, topDir, f, badmidkey, true);
|
||||
bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey, false);
|
||||
top = new StoreFile(this.fs, topPath, conf, cacheConf,
|
||||
StoreFile.BloomType.NONE,
|
||||
NoOpDataBlockEncoder.INSTANCE).createReader();
|
||||
|
@ -442,8 +438,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
}
|
||||
writer.close();
|
||||
|
||||
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf,
|
||||
DataBlockEncoding.NONE);
|
||||
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf, DataBlockEncoding.NONE);
|
||||
reader.loadFileInfo();
|
||||
reader.loadBloomfilter();
|
||||
|
||||
|
|
|
@ -21,9 +21,12 @@ package org.apache.hadoop.hbase.util;
|
|||
|
||||
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 java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -32,6 +35,7 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -46,6 +50,30 @@ import org.junit.experimental.categories.Category;
|
|||
*/
|
||||
@Category(MediumTests.class)
|
||||
public class TestFSUtils {
|
||||
@Test
|
||||
public void testVersion() throws DeserializationException, IOException {
|
||||
HBaseTestingUtility htu = new HBaseTestingUtility();
|
||||
final FileSystem fs = htu.getTestFileSystem();
|
||||
final Path rootdir = htu.getDataTestDir();
|
||||
assertNull(FSUtils.getVersion(fs, rootdir));
|
||||
// Write out old format version file. See if we can read it in and convert.
|
||||
Path versionFile = new Path(rootdir, HConstants.VERSION_FILE_NAME);
|
||||
FSDataOutputStream s = fs.create(versionFile);
|
||||
final String version = HConstants.FILE_SYSTEM_VERSION;
|
||||
s.writeUTF(version);
|
||||
s.close();
|
||||
assertTrue(fs.exists(versionFile));
|
||||
FileStatus [] status = fs.listStatus(versionFile);
|
||||
assertNotNull(status);
|
||||
assertTrue(status.length > 0);
|
||||
String newVersion = FSUtils.getVersion(fs, rootdir);
|
||||
assertEquals(version.length(), newVersion.length());
|
||||
assertEquals(version, newVersion);
|
||||
// File will have been converted. Exercise the pb format
|
||||
assertEquals(version, FSUtils.getVersion(fs, rootdir));
|
||||
FSUtils.checkVersion(fs, rootdir, true);
|
||||
}
|
||||
|
||||
@Test public void testIsHDFS() throws Exception {
|
||||
HBaseTestingUtility htu = new HBaseTestingUtility();
|
||||
htu.getConfiguration().setBoolean("dfs.support.append", false);
|
||||
|
|
|
@ -207,7 +207,7 @@ public class OfflineMetaRebuildTestCore {
|
|||
fs.mkdirs(p);
|
||||
Path riPath = new Path(p, HRegion.REGIONINFO_FILE);
|
||||
FSDataOutputStream out = fs.create(riPath);
|
||||
hri.write(out);
|
||||
out.write(hri.toDelimitedByteArray());
|
||||
out.close();
|
||||
|
||||
// add to meta.
|
||||
|
|
Loading…
Reference in New Issue