HBASE-7232 Remove HbaseMapWritable

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1415507 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-11-30 06:27:21 +00:00
parent cb8aca6e74
commit 813cd30de5
22 changed files with 906 additions and 317 deletions

View File

@ -0,0 +1,647 @@
// Generated by the protocol buffer compiler. DO NOT EDIT!
// source: HFile.proto
package org.apache.hadoop.hbase.protobuf.generated;
public final class HFileProtos {
private HFileProtos() {}
public static void registerAllExtensions(
com.google.protobuf.ExtensionRegistry registry) {
}
public interface FileInfoProtoOrBuilder
extends com.google.protobuf.MessageOrBuilder {
// repeated .BytesBytesPair mapEntry = 1;
java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair>
getMapEntryList();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getMapEntry(int index);
int getMapEntryCount();
java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>
getMapEntryOrBuilderList();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getMapEntryOrBuilder(
int index);
}
public static final class FileInfoProto extends
com.google.protobuf.GeneratedMessage
implements FileInfoProtoOrBuilder {
// Use FileInfoProto.newBuilder() to construct.
private FileInfoProto(Builder builder) {
super(builder);
}
private FileInfoProto(boolean noInit) {}
private static final FileInfoProto defaultInstance;
public static FileInfoProto getDefaultInstance() {
return defaultInstance;
}
public FileInfoProto getDefaultInstanceForType() {
return defaultInstance;
}
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.HFileProtos.internal_static_FileInfoProto_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.HFileProtos.internal_static_FileInfoProto_fieldAccessorTable;
}
// repeated .BytesBytesPair mapEntry = 1;
public static final int MAPENTRY_FIELD_NUMBER = 1;
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair> mapEntry_;
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair> getMapEntryList() {
return mapEntry_;
}
public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>
getMapEntryOrBuilderList() {
return mapEntry_;
}
public int getMapEntryCount() {
return mapEntry_.size();
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getMapEntry(int index) {
return mapEntry_.get(index);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getMapEntryOrBuilder(
int index) {
return mapEntry_.get(index);
}
private void initFields() {
mapEntry_ = java.util.Collections.emptyList();
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
if (isInitialized != -1) return isInitialized == 1;
for (int i = 0; i < getMapEntryCount(); i++) {
if (!getMapEntry(i).isInitialized()) {
memoizedIsInitialized = 0;
return false;
}
}
memoizedIsInitialized = 1;
return true;
}
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
for (int i = 0; i < mapEntry_.size(); i++) {
output.writeMessage(1, mapEntry_.get(i));
}
getUnknownFields().writeTo(output);
}
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
size = 0;
for (int i = 0; i < mapEntry_.size(); i++) {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, mapEntry_.get(i));
}
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.HFileProtos.FileInfoProto)) {
return super.equals(obj);
}
org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto other = (org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto) obj;
boolean result = true;
result = result && getMapEntryList()
.equals(other.getMapEntryList());
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
}
@java.lang.Override
public int hashCode() {
int hash = 41;
hash = (19 * hash) + getDescriptorForType().hashCode();
if (getMapEntryCount() > 0) {
hash = (37 * hash) + MAPENTRY_FIELD_NUMBER;
hash = (53 * hash) + getMapEntryList().hashCode();
}
hash = (29 * hash) + getUnknownFields().hashCode();
return hash;
}
public static org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto 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.HFileProtos.FileInfoProto parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto 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.HFileProtos.FileInfoProto parseFrom(java.io.InputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto 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.HFileProtos.FileInfoProto 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.HFileProtos.FileInfoProto 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.HFileProtos.FileInfoProto parseFrom(
com.google.protobuf.CodedInputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto 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.HFileProtos.FileInfoProto 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.HFileProtos.FileInfoProtoOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.HFileProtos.internal_static_FileInfoProto_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.HFileProtos.internal_static_FileInfoProto_fieldAccessorTable;
}
// Construct using org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto.newBuilder()
private Builder() {
maybeForceBuilderInitialization();
}
private Builder(BuilderParent parent) {
super(parent);
maybeForceBuilderInitialization();
}
private void maybeForceBuilderInitialization() {
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
getMapEntryFieldBuilder();
}
}
private static Builder create() {
return new Builder();
}
public Builder clear() {
super.clear();
if (mapEntryBuilder_ == null) {
mapEntry_ = java.util.Collections.emptyList();
bitField0_ = (bitField0_ & ~0x00000001);
} else {
mapEntryBuilder_.clear();
}
return this;
}
public Builder clone() {
return create().mergeFrom(buildPartial());
}
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto.getDescriptor();
}
public org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto getDefaultInstanceForType() {
return org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto.getDefaultInstance();
}
public org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto build() {
org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
}
return result;
}
private org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(
result).asInvalidProtocolBufferException();
}
return result;
}
public org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto buildPartial() {
org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto result = new org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto(this);
int from_bitField0_ = bitField0_;
if (mapEntryBuilder_ == null) {
if (((bitField0_ & 0x00000001) == 0x00000001)) {
mapEntry_ = java.util.Collections.unmodifiableList(mapEntry_);
bitField0_ = (bitField0_ & ~0x00000001);
}
result.mapEntry_ = mapEntry_;
} else {
result.mapEntry_ = mapEntryBuilder_.build();
}
onBuilt();
return result;
}
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto) {
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto)other);
} else {
super.mergeFrom(other);
return this;
}
}
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto other) {
if (other == org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto.getDefaultInstance()) return this;
if (mapEntryBuilder_ == null) {
if (!other.mapEntry_.isEmpty()) {
if (mapEntry_.isEmpty()) {
mapEntry_ = other.mapEntry_;
bitField0_ = (bitField0_ & ~0x00000001);
} else {
ensureMapEntryIsMutable();
mapEntry_.addAll(other.mapEntry_);
}
onChanged();
}
} else {
if (!other.mapEntry_.isEmpty()) {
if (mapEntryBuilder_.isEmpty()) {
mapEntryBuilder_.dispose();
mapEntryBuilder_ = null;
mapEntry_ = other.mapEntry_;
bitField0_ = (bitField0_ & ~0x00000001);
mapEntryBuilder_ =
com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
getMapEntryFieldBuilder() : null;
} else {
mapEntryBuilder_.addAllMessages(other.mapEntry_);
}
}
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
public final boolean isInitialized() {
for (int i = 0; i < getMapEntryCount(); i++) {
if (!getMapEntry(i).isInitialized()) {
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: {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.newBuilder();
input.readMessage(subBuilder, extensionRegistry);
addMapEntry(subBuilder.buildPartial());
break;
}
}
}
}
private int bitField0_;
// repeated .BytesBytesPair mapEntry = 1;
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair> mapEntry_ =
java.util.Collections.emptyList();
private void ensureMapEntryIsMutable() {
if (!((bitField0_ & 0x00000001) == 0x00000001)) {
mapEntry_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair>(mapEntry_);
bitField0_ |= 0x00000001;
}
}
private com.google.protobuf.RepeatedFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> mapEntryBuilder_;
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair> getMapEntryList() {
if (mapEntryBuilder_ == null) {
return java.util.Collections.unmodifiableList(mapEntry_);
} else {
return mapEntryBuilder_.getMessageList();
}
}
public int getMapEntryCount() {
if (mapEntryBuilder_ == null) {
return mapEntry_.size();
} else {
return mapEntryBuilder_.getCount();
}
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getMapEntry(int index) {
if (mapEntryBuilder_ == null) {
return mapEntry_.get(index);
} else {
return mapEntryBuilder_.getMessage(index);
}
}
public Builder setMapEntry(
int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair value) {
if (mapEntryBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
}
ensureMapEntryIsMutable();
mapEntry_.set(index, value);
onChanged();
} else {
mapEntryBuilder_.setMessage(index, value);
}
return this;
}
public Builder setMapEntry(
int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) {
if (mapEntryBuilder_ == null) {
ensureMapEntryIsMutable();
mapEntry_.set(index, builderForValue.build());
onChanged();
} else {
mapEntryBuilder_.setMessage(index, builderForValue.build());
}
return this;
}
public Builder addMapEntry(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair value) {
if (mapEntryBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
}
ensureMapEntryIsMutable();
mapEntry_.add(value);
onChanged();
} else {
mapEntryBuilder_.addMessage(value);
}
return this;
}
public Builder addMapEntry(
int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair value) {
if (mapEntryBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
}
ensureMapEntryIsMutable();
mapEntry_.add(index, value);
onChanged();
} else {
mapEntryBuilder_.addMessage(index, value);
}
return this;
}
public Builder addMapEntry(
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) {
if (mapEntryBuilder_ == null) {
ensureMapEntryIsMutable();
mapEntry_.add(builderForValue.build());
onChanged();
} else {
mapEntryBuilder_.addMessage(builderForValue.build());
}
return this;
}
public Builder addMapEntry(
int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) {
if (mapEntryBuilder_ == null) {
ensureMapEntryIsMutable();
mapEntry_.add(index, builderForValue.build());
onChanged();
} else {
mapEntryBuilder_.addMessage(index, builderForValue.build());
}
return this;
}
public Builder addAllMapEntry(
java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair> values) {
if (mapEntryBuilder_ == null) {
ensureMapEntryIsMutable();
super.addAll(values, mapEntry_);
onChanged();
} else {
mapEntryBuilder_.addAllMessages(values);
}
return this;
}
public Builder clearMapEntry() {
if (mapEntryBuilder_ == null) {
mapEntry_ = java.util.Collections.emptyList();
bitField0_ = (bitField0_ & ~0x00000001);
onChanged();
} else {
mapEntryBuilder_.clear();
}
return this;
}
public Builder removeMapEntry(int index) {
if (mapEntryBuilder_ == null) {
ensureMapEntryIsMutable();
mapEntry_.remove(index);
onChanged();
} else {
mapEntryBuilder_.remove(index);
}
return this;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder getMapEntryBuilder(
int index) {
return getMapEntryFieldBuilder().getBuilder(index);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getMapEntryOrBuilder(
int index) {
if (mapEntryBuilder_ == null) {
return mapEntry_.get(index); } else {
return mapEntryBuilder_.getMessageOrBuilder(index);
}
}
public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>
getMapEntryOrBuilderList() {
if (mapEntryBuilder_ != null) {
return mapEntryBuilder_.getMessageOrBuilderList();
} else {
return java.util.Collections.unmodifiableList(mapEntry_);
}
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder addMapEntryBuilder() {
return getMapEntryFieldBuilder().addBuilder(
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.getDefaultInstance());
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder addMapEntryBuilder(
int index) {
return getMapEntryFieldBuilder().addBuilder(
index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.getDefaultInstance());
}
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder>
getMapEntryBuilderList() {
return getMapEntryFieldBuilder().getBuilderList();
}
private com.google.protobuf.RepeatedFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>
getMapEntryFieldBuilder() {
if (mapEntryBuilder_ == null) {
mapEntryBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>(
mapEntry_,
((bitField0_ & 0x00000001) == 0x00000001),
getParentForChildren(),
isClean());
mapEntry_ = null;
}
return mapEntryBuilder_;
}
// @@protoc_insertion_point(builder_scope:FileInfoProto)
}
static {
defaultInstance = new FileInfoProto(true);
defaultInstance.initFields();
}
// @@protoc_insertion_point(class_scope:FileInfoProto)
}
private static com.google.protobuf.Descriptors.Descriptor
internal_static_FileInfoProto_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_FileInfoProto_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\013HFile.proto\032\013hbase.proto\"2\n\rFileInfoPr" +
"oto\022!\n\010mapEntry\030\001 \003(\0132\017.BytesBytesPairBA" +
"\n*org.apache.hadoop.hbase.protobuf.gener" +
"atedB\013HFileProtosH\001\210\001\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_FileInfoProto_descriptor =
getDescriptor().getMessageTypes().get(0);
internal_static_FileInfoProto_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_FileInfoProto_descriptor,
new java.lang.String[] { "MapEntry", },
org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto.class,
org.apache.hadoop.hbase.protobuf.generated.HFileProtos.FileInfoProto.Builder.class);
return null;
}
};
com.google.protobuf.Descriptors.FileDescriptor
.internalBuildGeneratedFileFrom(descriptorData,
new com.google.protobuf.Descriptors.FileDescriptor[] {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(),
}, assigner);
}
// @@protoc_insertion_point(outer_class_scope)
}

View File

@ -0,0 +1,29 @@
/**
* 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.
*/
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
option java_outer_classname = "HFileProtos";
option java_generic_services = true;
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
import "hbase.proto";
// Map of name/values
message FileInfoProto {
repeated BytesBytesPair mapEntry = 1;
}

View File

@ -1,222 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.io;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Collection;
import java.util.Comparator;
import java.util.Map;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.util.ReflectionUtils;
/**
* A Writable Map.
* Like {@link org.apache.hadoop.io.MapWritable} but dumb. It will fail
* if passed a value type that it has not already been told about. Its been
* primed with hbase Writables and byte []. Keys are always byte arrays.
*
* @param <K> <byte []> key TODO: Parameter K is never used, could be removed.
* @param <V> value Expects a Writable or byte [].
*/
@InterfaceAudience.Private
public class HbaseMapWritable <K,V>
implements SortedMap<byte[],V>, Configurable, Writable, CodeToClassAndBack{
private AtomicReference<Configuration> conf = null;
protected SortedMap<byte [], V> instance = null;
/**
* The default contructor where a TreeMap is used
**/
public HbaseMapWritable(){
this (new TreeMap<byte [], V>(Bytes.BYTES_COMPARATOR));
}
/**
* Contructor where another SortedMap can be used
*
* @param map the SortedMap to be used
*/
public HbaseMapWritable(SortedMap<byte[], V> map){
conf = new AtomicReference<Configuration>();
instance = map;
}
/** @return the conf */
public Configuration getConf() {
return conf.get();
}
/** @param conf the conf to set */
public void setConf(Configuration conf) {
this.conf.set(conf);
}
public void clear() {
instance.clear();
}
public boolean containsKey(Object key) {
return instance.containsKey(key);
}
public boolean containsValue(Object value) {
return instance.containsValue(value);
}
public Set<Entry<byte [], V>> entrySet() {
return instance.entrySet();
}
public V get(Object key) {
return instance.get(key);
}
public boolean isEmpty() {
return instance.isEmpty();
}
public Set<byte []> keySet() {
return instance.keySet();
}
public int size() {
return instance.size();
}
public Collection<V> values() {
return instance.values();
}
public void putAll(Map<? extends byte [], ? extends V> m) {
this.instance.putAll(m);
}
public V remove(Object key) {
return this.instance.remove(key);
}
public V put(byte [] key, V value) {
return this.instance.put(key, value);
}
public Comparator<? super byte[]> comparator() {
return this.instance.comparator();
}
public byte[] firstKey() {
return this.instance.firstKey();
}
public SortedMap<byte[], V> headMap(byte[] toKey) {
return this.instance.headMap(toKey);
}
public byte[] lastKey() {
return this.instance.lastKey();
}
public SortedMap<byte[], V> subMap(byte[] fromKey, byte[] toKey) {
return this.instance.subMap(fromKey, toKey);
}
public SortedMap<byte[], V> tailMap(byte[] fromKey) {
return this.instance.tailMap(fromKey);
}
// Writable
/** @return the Class class for the specified id */
@SuppressWarnings("boxing")
protected Class<?> getClass(byte id) {
return CODE_TO_CLASS.get(id);
}
/** @return the id for the specified Class */
@SuppressWarnings("boxing")
protected byte getId(Class<?> clazz) {
Byte b = CLASS_TO_CODE.get(clazz);
if (b == null) {
throw new NullPointerException("Nothing for : " + clazz);
}
return b;
}
/**
* @see java.lang.Object#toString()
*/
@Override
public String toString() {
return this.instance.toString();
}
public void write(DataOutput out) throws IOException {
// Write out the number of entries in the map
out.writeInt(this.instance.size());
// Then write out each key/value pair
for (Map.Entry<byte [], V> e: instance.entrySet()) {
Bytes.writeByteArray(out, e.getKey());
Byte id = getId(e.getValue().getClass());
out.writeByte(id);
Object value = e.getValue();
if (value instanceof byte []) {
Bytes.writeByteArray(out, (byte [])value);
} else {
((Writable)value).write(out);
}
}
}
@SuppressWarnings("unchecked")
public void readFields(DataInput in) throws IOException {
// First clear the map. Otherwise we will just accumulate
// entries every time this method is called.
this.instance.clear();
// Read the number of entries in the map
int entries = in.readInt();
// Then read each key/value pair
for (int i = 0; i < entries; i++) {
byte [] key = Bytes.readByteArray(in);
byte id = in.readByte();
Class clazz = getClass(id);
V value = null;
if (clazz.equals(byte [].class)) {
byte [] bytes = Bytes.readByteArray(in);
value = (V)bytes;
} else {
Writable w = (Writable)ReflectionUtils.
newInstance(clazz, getConf());
w.readFields(in);
value = (V)w;
}
this.instance.put(key, value);
}
}
}

View File

@ -167,7 +167,7 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
addToMap(Text.class, code++);
addToMap(Writable.class, code++);
addToMap(Writable [].class, code++);
addToMap(HbaseMapWritable.class, code++);
code++; // Removed
addToMap(NullInstance.class, code++);
// Hbase types

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FSDataInputStream;

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hbase.io.hfile;
import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -134,7 +134,7 @@ public abstract class AbstractHFileWriter implements HFile.Writer {
*/
protected void finishFileInfo() throws IOException {
if (lastKeyBuffer != null) {
// Make a copy. The copy is stuffed into HMapWritable. Needs a clean
// Make a copy. The copy is stuffed into our fileinfo map. Needs a clean
// byte buffer. Won't take a tuple.
fileInfo.append(FileInfo.LASTKEY, Arrays.copyOfRange(lastKeyBuffer,
lastKeyOffset, lastKeyOffset + lastKeyLength), false);
@ -175,8 +175,8 @@ public abstract class AbstractHFileWriter implements HFile.Writer {
* @param out the data output to write the file info to
* @throws IOException
*/
protected final void writeFileInfo(FixedFileTrailer trailer, DataOutput out)
throws IOException {
protected final void writeFileInfo(FixedFileTrailer trailer, DataOutputStream out)
throws IOException {
trailer.setFileInfoOffset(outputStream.getPos());
finishFileInfo();
fileInfo.write(out);

View File

@ -18,14 +18,22 @@
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.ByteArrayInputStream;
import java.io.Closeable;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.SequenceInputStream;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
@ -46,9 +54,12 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
import org.apache.hadoop.hbase.util.BloomFilterWriter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ChecksumType;
@ -58,6 +69,7 @@ import org.apache.hadoop.io.Writable;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
/**
* File format for hbase.
@ -145,7 +157,8 @@ public class HFile {
/** Minimum supported HFile format version */
public static final int MIN_FORMAT_VERSION = 1;
/** Maximum supported HFile format version */
/** Maximum supported HFile format version
*/
public static final int MAX_FORMAT_VERSION = 2;
/** Default compression name: none. */
@ -282,14 +295,19 @@ public class HFile {
/** @return the path to this {@link HFile} */
Path getPath();
void appendMetaBlock(String bloomFilterMetaKey, Writable metaWriter);
/**
* Adds an inline block writer such as a multi-level block index writer or
* a compound Bloom filter writer.
*/
void addInlineBlockWriter(InlineBlockWriter bloomWriter);
// The below three methods take Writables. We'd like to undo Writables but undoing the below would be pretty
// painful. Could take a byte [] or a Message but we want to be backward compatible around hfiles so would need
// to map between Message and Writable or byte [] and current Writable serialization. This would be a bit of work
// to little gain. Thats my thinking at moment. St.Ack 20121129
void appendMetaBlock(String bloomFilterMetaKey, Writable metaWriter);
/**
* Store general Bloom filter in the file. This does not deal with Bloom filter
* internals but is necessary, since Bloom filters are stored differently
@ -642,19 +660,21 @@ public class HFile {
DataBlockEncoding.NONE, null);
}
/*
/**
* Metadata for this file. Conjured by the writer. Read in by the reader.
*/
static class FileInfo extends HbaseMapWritable<byte [], byte []> {
static class FileInfo implements SortedMap<byte [], byte []> {
static final String RESERVED_PREFIX = "hfile.";
static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
static final byte [] AVG_KEY_LEN =
Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
static final byte [] AVG_VALUE_LEN =
Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
static final byte [] COMPARATOR =
Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
static final byte [] AVG_KEY_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
static final byte [] AVG_VALUE_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
static final byte [] COMPARATOR = Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
private final SortedMap<byte [], byte []> map = new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
public FileInfo() {
super();
}
/**
* Append the given key/value pair to the file info, optionally checking the
@ -680,6 +700,167 @@ public class HFile {
return this;
}
public void clear() {
this.map.clear();
}
public Comparator<? super byte[]> comparator() {
return map.comparator();
}
public boolean containsKey(Object key) {
return map.containsKey(key);
}
public boolean containsValue(Object value) {
return map.containsValue(value);
}
public Set<java.util.Map.Entry<byte[], byte[]>> entrySet() {
return map.entrySet();
}
public boolean equals(Object o) {
return map.equals(o);
}
public byte[] firstKey() {
return map.firstKey();
}
public byte[] get(Object key) {
return map.get(key);
}
public int hashCode() {
return map.hashCode();
}
public SortedMap<byte[], byte[]> headMap(byte[] toKey) {
return this.map.headMap(toKey);
}
public boolean isEmpty() {
return map.isEmpty();
}
public Set<byte[]> keySet() {
return map.keySet();
}
public byte[] lastKey() {
return map.lastKey();
}
public byte[] put(byte[] key, byte[] value) {
return this.map.put(key, value);
}
public void putAll(Map<? extends byte[], ? extends byte[]> m) {
this.map.putAll(m);
}
public byte[] remove(Object key) {
return this.map.remove(key);
}
public int size() {
return map.size();
}
public SortedMap<byte[], byte[]> subMap(byte[] fromKey, byte[] toKey) {
return this.map.subMap(fromKey, toKey);
}
public SortedMap<byte[], byte[]> tailMap(byte[] fromKey) {
return this.map.tailMap(fromKey);
}
public Collection<byte[]> values() {
return map.values();
}
/**
* Write out this instance on the passed in <code>out</code> stream.
* We write it as a protobuf.
* @param out
* @throws IOException
* @see {@link #read(DataInputStream)}
*/
void write(final DataOutputStream out) throws IOException {
HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder();
for (Map.Entry<byte [], byte[]> e: this.map.entrySet()) {
HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder();
bbpBuilder.setFirst(ByteString.copyFrom(e.getKey()));
bbpBuilder.setSecond(ByteString.copyFrom(e.getValue()));
builder.addMapEntry(bbpBuilder.build());
}
out.write(ProtobufUtil.PB_MAGIC);
builder.build().writeDelimitedTo(out);
}
/**
* Populate this instance with what we find on the passed in <code>in</code> stream.
* Can deserialize protobuf of old Writables format.
* @param in
* @throws IOException
* @see {@link #write(DataOutputStream)}
*/
void read(final DataInputStream in) throws IOException {
// This code is tested over in TestHFileReaderV1 where we read an old hfile w/ this new code.
int pblen = ProtobufUtil.lengthOfPBMagic();
byte [] pbuf = new byte[pblen];
if (in.markSupported()) in.mark(pblen);
int read = in.read(pbuf);
if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
parsePB(HFileProtos.FileInfoProto.parseDelimitedFrom(in));
} else {
if (in.markSupported()) {
in.reset();
parseWritable(in);
} else {
// We cannot use BufferedInputStream, it consumes more than we read from the underlying IS
ByteArrayInputStream bais = new ByteArrayInputStream(pbuf);
SequenceInputStream sis = new SequenceInputStream(bais, in); // Concatenate input streams
// TODO: Am I leaking anything here wrapping the passed in stream? We are not calling close on the wrapped
// streams but they should be let go after we leave this context? I see that we keep a reference to the
// passed in inputstream but since we no longer have a reference to this after we leave, we should be ok.
parseWritable(new DataInputStream(sis));
}
}
}
/** Now parse the old Writable format. It was a list of Map entries. Each map entry was a key and a value of
* a byte []. The old map format had a byte before each entry that held a code which was short for the key or
* value type. We know it was a byte [] so in below we just read and dump it.
* @throws IOException
*/
void parseWritable(final DataInputStream in) throws IOException {
// First clear the map. Otherwise we will just accumulate entries every time this method is called.
this.map.clear();
// Read the number of entries in the map
int entries = in.readInt();
// Then read each key/value pair
for (int i = 0; i < entries; i++) {
byte [] key = Bytes.readByteArray(in);
// We used to read a byte that encoded the class type. Read and ignore it because it is always byte [] in hfile
in.readByte();
byte [] value = Bytes.readByteArray(in);
this.map.put(key, value);
}
}
/**
* Fill our map with content of the pb we read off disk
* @param fip protobuf message to read
*/
void parsePB(final HFileProtos.FileInfoProto fip) {
this.map.clear();
for (BytesBytesPair pair: fip.getMapEntryList()) {
this.map.put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
}
}
}
/** Return true if the given file info key is reserved for internal use. */
@ -761,5 +942,4 @@ public class HFile {
+ MAX_FORMAT_VERSION + ")");
}
}
}

View File

@ -46,9 +46,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ChecksumType;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.CompoundBloomFilter;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Writable;
import com.google.common.base.Preconditions;
@ -370,20 +368,6 @@ public class HFileBlock implements Cacheable {
return dupBuf;
}
/**
* Deserializes fields of the given writable using the data portion of this
* block. Does not check that all the block data has been read.
*/
void readInto(Writable w) throws IOException {
Preconditions.checkNotNull(w);
if (Writables.getWritable(buf.array(), buf.arrayOffset() + headerSize(),
buf.limit() - headerSize(), w) == null) {
throw new IOException("Failed to deserialize block " + this + " into a "
+ w.getClass().getSimpleName());
}
}
private void sanityCheckAssertion(long valueFromBuf, long valueFromField,
String fieldName) throws IOException {
if (valueFromBuf != valueFromField) {
@ -634,9 +618,6 @@ public class HFileBlock implements Cacheable {
/** Writer state. Used to ensure the correct usage protocol. */
private State state = State.INIT;
/** Compression algorithm for all blocks this instance writes. */
private final Compression.Algorithm compressAlgo;
/** Data block encoder used for data blocks */
private final HFileDataBlockEncoder dataBlockEncoder;
@ -720,8 +701,6 @@ public class HFileBlock implements Cacheable {
public Writer(Compression.Algorithm compressionAlgorithm,
HFileDataBlockEncoder dataBlockEncoder, boolean includesMemstoreTS,
ChecksumType checksumType, int bytesPerChecksum) {
compressAlgo = compressionAlgorithm == null ? Compression.Algorithm.NONE :
compressionAlgorithm;
this.dataBlockEncoder = dataBlockEncoder != null
? dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
defaultBlockEncodingCtx =

View File

@ -1127,7 +1127,7 @@ public class HFileBlockIndex {
* or null.
*/
@Override
public boolean cacheOnWrite() {
public boolean getCacheOnWrite() {
return blockCache != null;
}

View File

@ -94,7 +94,7 @@ public class HFileReaderV1 extends AbstractHFileReader {
// Read in the fileinfo and get what we need from it.
istream.seek(trailer.getFileInfoOffset());
fileInfo = new FileInfo();
fileInfo.readFields(istream);
fileInfo.read(istream);
lastKey = fileInfo.get(FileInfo.LASTKEY);
avgKeyLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_KEY_LEN));
avgValueLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_VALUE_LEN));

View File

@ -77,7 +77,9 @@ public class HFileReaderV2 extends AbstractHFileReader {
static final int MIN_MINOR_VERSION = 0;
/** Maximum minor version supported by this HFile format */
static final int MAX_MINOR_VERSION = 1;
// We went to version 2 when we moved to pb'ing the fileinfo trailer on the file. This version can read Writables
// version 1 too.
static final int MAX_MINOR_VERSION = 2;
/**
* Opens a HFile. You must load the index before you can use it by calling
@ -135,7 +137,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
// File info
fileInfo = new FileInfo();
fileInfo.readFields(blockIter.nextBlockWithBlockType(BlockType.FILE_INFO).getByteStream());
fileInfo.read(blockIter.nextBlockWithBlockType(BlockType.FILE_INFO).getByteStream());
lastKey = fileInfo.get(FileInfo.LASTKEY);
avgKeyLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_KEY_LEN));
avgValueLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_VALUE_LEN));

View File

@ -81,7 +81,6 @@ public class HFileWriterV1 extends AbstractHFileWriter {
// Additional byte array output stream used to fill block cache
private ByteArrayOutputStream baos;
private DataOutputStream baosDos;
private int blockNumber = 0;
static class WriterFactoryV1 extends HFile.WriterFactory {
WriterFactoryV1(Configuration conf, CacheConfig cacheConf) {
@ -161,7 +160,6 @@ public class HFileWriterV1 extends AbstractHFileWriter {
block.getBlockType()), block);
baosDos.close();
}
blockNumber++;
}
/**

View File

@ -189,7 +189,7 @@ public class HFileWriterV2 extends AbstractHFileWriter {
for (InlineBlockWriter ibw : inlineBlockWriters) {
while (ibw.shouldWriteBlock(closing)) {
long offset = outputStream.getPos();
boolean cacheThisBlock = ibw.cacheOnWrite();
boolean cacheThisBlock = ibw.getCacheOnWrite();
ibw.writeInlineBlock(fsBlockWriter.startWriting(
ibw.getInlineBlockType()));
fsBlockWriter.writeHeaderAndData(outputStream);
@ -463,5 +463,4 @@ public class HFileWriterV2 extends AbstractHFileWriter {
}
});
}
}
}

View File

@ -69,5 +69,5 @@ public interface InlineBlockWriter {
/**
* @return true if inline blocks produced by this writer should be cached
*/
boolean cacheOnWrite();
boolean getCacheOnWrite();
}

View File

@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.MutationSerialization;
import org.apache.hadoop.hbase.mapreduce.ResultSerialization;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.InputFormat;

View File

@ -32,7 +32,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.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;

View File

@ -168,7 +168,7 @@ public final class ProtobufUtil {
* For example, all znode content is protobuf messages with the below magic
* for preamble.
*/
static final byte [] PB_MAGIC = new byte [] {'P', 'B', 'U', 'F'};
public static final byte [] PB_MAGIC = new byte [] {'P', 'B', 'U', 'F'};
private static final String PB_MAGIC_STR = Bytes.toString(PB_MAGIC);
/**

View File

@ -271,8 +271,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
}
@Override
public boolean cacheOnWrite() {
public boolean getCacheOnWrite() {
return cacheOnWrite;
}
}
}

View File

@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.RowFilter;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@ -78,19 +77,6 @@ public class TestSerialization {
assertTrue(KeyValue.COMPARATOR.compare(original, newone) == 0);
}
@SuppressWarnings("unchecked")
@Test public void testHbaseMapWritable() throws Exception {
HbaseMapWritable<byte [], byte []> hmw =
new HbaseMapWritable<byte[], byte[]>();
hmw.put("key".getBytes(), "value".getBytes());
byte [] bytes = Writables.getBytes(hmw);
hmw = (HbaseMapWritable<byte[], byte[]>)
Writables.getWritable(bytes, new HbaseMapWritable<byte [], byte []>());
assertTrue(hmw.size() == 1);
assertTrue(Bytes.equals("value".getBytes(), hmw.get("key".getBytes())));
}
@Test public void testTableDescriptor() throws Exception {
final String name = "testTableDescriptor";
HTableDescriptor htd = createTableDescriptor(name);

View File

@ -475,7 +475,7 @@ public class TestHbaseObjectWritable extends TestCase {
assertEquals(13,HbaseObjectWritable.getClassCode(Text.class).intValue());
assertEquals(14,HbaseObjectWritable.getClassCode(Writable.class).intValue());
assertEquals(15,HbaseObjectWritable.getClassCode(Writable [].class).intValue());
assertEquals(16,HbaseObjectWritable.getClassCode(HbaseMapWritable.class).intValue());
// 16 Removed
// 17 is NullInstance which isn't visible from here
// Hbase types

View File

@ -19,9 +19,11 @@
package org.apache.hadoop.hbase.io.hfile;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.net.URL;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -30,13 +32,10 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
import org.mortbay.log.Log;
@Category(SmallTests.class)
public class TestHFileReaderV1 {
@ -90,7 +89,4 @@ public class TestHFileReaderV1 {
}
reader.close();
}
}
}

View File

@ -19,7 +19,9 @@
package org.apache.hadoop.hbase.io.hfile;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
@ -35,11 +37,14 @@ 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.*;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableUtils;
@ -70,14 +75,12 @@ public class TestHFileWriterV2 {
@Test
public void testHFileFormatV2() throws IOException {
Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
"testHFileFormatV2");
Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), "testHFileFormatV2");
final Compression.Algorithm compressAlgo = Compression.Algorithm.GZ;
final int entryCount = 10000;
writeDataAndReadFromHFile(hfilePath, compressAlgo, entryCount, false);
}
@Test
public void testMidKeyInHFile() throws IOException{
Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
@ -98,9 +101,6 @@ public class TestHFileWriterV2 {
.withComparator(KeyValue.KEY_COMPARATOR)
.create();
long totalKeyLength = 0;
long totalValueLength = 0;
Random rand = new Random(9713312); // Just a fixed seed.
List<byte[]> keys = new ArrayList<byte[]>();
@ -113,9 +113,6 @@ public class TestHFileWriterV2 {
byte[] valueBytes = randomValue(rand);
writer.append(keyBytes, valueBytes);
totalKeyLength += keyBytes.length;
totalValueLength += valueBytes.length;
keys.add(keyBytes);
values.add(valueBytes);
}
@ -172,7 +169,7 @@ public class TestHFileWriterV2 {
trailer.getMetaIndexCount());
// File info
FileInfo fileInfo = new FileInfo();
fileInfo.readFields(blockIter.nextBlockWithBlockType(BlockType.FILE_INFO).getByteStream());
fileInfo.read(blockIter.nextBlockWithBlockType(BlockType.FILE_INFO).getByteStream());
byte [] keyValueFormatVersion = fileInfo.get(
HFileWriterV2.KEY_VALUE_VERSION);
boolean includeMemstoreTS = keyValueFormatVersion != null &&
@ -233,7 +230,10 @@ public class TestHFileWriterV2 {
HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false);
assertEquals(BlockType.META, block.getBlockType());
Text t = new Text();
block.readInto(t);
ByteBuffer buf = block.getBufferWithoutHeader();
if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), t) == null) {
throw new IOException("Failed to deserialize block " + this + " into a " + t.getClass().getSimpleName());
}
Text expectedText =
(metaCounter == 0 ? new Text("Paris") : metaCounter == 1 ? new Text(
"Moscow") : new Text("Washington, D.C."));