diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml index 85bb7ad..964cb5d 100644 --- a/hbase-common/pom.xml +++ b/hbase-common/pom.xml @@ -67,6 +67,10 @@ guava + com.google.protobuf + protobuf-java + + commons-logging commons-logging @@ -77,7 +81,7 @@ org.slf4j slf4j-log4j12 - + diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java index 04bacb4..f6d5c3f 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.io.RawComparator; import org.apache.hbase.Cell; -import org.apache.hbase.cell.CellComparator; +import org.apache.hbase.CellComparator; import com.google.common.primitives.Longs; @@ -915,7 +915,7 @@ public class KeyValue implements Cell, HeapSize { return "empty"; } return keyToString(this.bytes, this.offset + ROW_OFFSET, getKeyLength()) + - "/vlen=" + getValueLength() + "/ts=" + memstoreTS; + "/vlen=" + getValueLength() + "/mvcc=" + memstoreTS; } /** @@ -2299,8 +2299,10 @@ public class KeyValue implements Cell, HeapSize { } /** - * @param in Where to read bytes from - * @return KeyValue created by deserializing from in + * @param in Where to read bytes from. Creates a byte array to hold the KeyValue + * backing bytes copied from the steam. + * @return KeyValue created by deserializing from in OR if we find a length + * of zero, we will return null which can be useful marking a stream as done. * @throws IOException */ public static KeyValue create(final DataInput in) throws IOException { @@ -2311,7 +2313,8 @@ public class KeyValue implements Cell, HeapSize { * Create a KeyValue reading length from in * @param length * @param in - * @return Created KeyValue + * @return Created KeyValue OR if we find a length of zero, we will return null which + * can be useful marking a stream as done. * @throws IOException */ public static KeyValue create(int length, final DataInput in) throws IOException { diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java index c2052b0..bb8923c 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java @@ -26,7 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.IterableUtils; import org.apache.hadoop.hbase.util.Strings; -import org.apache.hbase.cell.CellComparator; +import org.apache.hbase.CellComparator; import com.google.common.collect.Lists; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTool.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTool.java index e6297e2..9a5dca5 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTool.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTool.java @@ -21,14 +21,13 @@ package org.apache.hadoop.hbase; import java.nio.ByteBuffer; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.ByteRange; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.IterableUtils; import org.apache.hadoop.io.WritableUtils; import org.apache.hbase.Cell; -import org.apache.hbase.cell.CellTool; +import org.apache.hbase.CellTool; /** * static convenience methods for dealing with KeyValues and collections of KeyValues diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java index b2ce35a..07b85d3 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java @@ -31,7 +31,6 @@ import org.apache.hadoop.io.RawComparator; *
  • the KeyValues are stored sorted by key
  • *
  • we know the structure of KeyValue
  • *
  • the values are always iterated forward from beginning of block
  • - *
  • knowledge of Key Value format
  • * * It is designed to work fast enough to be feasible as in memory compression. * diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/LoadTestKVGenerator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/LoadTestKVGenerator.java index 5a64a1b..6a30ca3 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/LoadTestKVGenerator.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/LoadTestKVGenerator.java @@ -16,7 +16,6 @@ */ package org.apache.hadoop.hbase.util.test; -import java.util.Map; import java.util.Random; import org.apache.hadoop.hbase.util.Bytes; diff --git a/hbase-common/src/main/java/org/apache/hbase/Cell.java b/hbase-common/src/main/java/org/apache/hbase/Cell.java index 044b287..532f8ae 100644 --- a/hbase-common/src/main/java/org/apache/hbase/Cell.java +++ b/hbase-common/src/main/java/org/apache/hbase/Cell.java @@ -20,7 +20,6 @@ package org.apache.hbase; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hbase.cell.CellTool; /** diff --git a/hbase-common/src/main/java/org/apache/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hbase/CellComparator.java new file mode 100644 index 0000000..f6a3c25 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hbase/CellComparator.java @@ -0,0 +1,177 @@ +/* + * 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.hbase; + +import java.io.Serializable; +import java.util.Comparator; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; + +import com.google.common.primitives.Longs; + +/** + * Compare two traditional HBase cells. + * + * Note: This comparator is not valid for -ROOT- and .META. tables. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class CellComparator implements Comparator, Serializable{ + private static final long serialVersionUID = -8760041766259623329L; + + @Override + public int compare(Cell a, Cell b) { + return compareStatic(a, b); + } + + + public static int compareStatic(Cell a, Cell b) { + //row + int c = Bytes.compareTo( + a.getRowArray(), a.getRowOffset(), a.getRowLength(), + b.getRowArray(), b.getRowOffset(), b.getRowLength()); + if (c != 0) return c; + + //family + c = Bytes.compareTo( + a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(), + b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength()); + if (c != 0) return c; + + //qualifier + c = Bytes.compareTo( + a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(), + b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength()); + if (c != 0) return c; + + //timestamp: later sorts first + c = -Longs.compare(a.getTimestamp(), b.getTimestamp()); + if (c != 0) return c; + + //type + c = (0xff & a.getTypeByte()) - (0xff & b.getTypeByte()); + if (c != 0) return c; + + //mvccVersion: later sorts first + return -Longs.compare(a.getMvccVersion(), b.getMvccVersion()); + } + + + /**************** equals ****************************/ + + public static boolean equals(Cell a, Cell b){ + if (!areKeyLengthsEqual(a, b)) { + return false; + } + //TODO compare byte[]'s in reverse since later bytes more likely to differ + return 0 == compareStatic(a, b); + } + + public static boolean equalsRow(Cell a, Cell b){ + if(!areRowLengthsEqual(a, b)){ + return false; + } + return 0 == Bytes.compareTo( + a.getRowArray(), a.getRowOffset(), a.getRowLength(), + b.getRowArray(), b.getRowOffset(), b.getRowLength()); + } + + + /********************* hashCode ************************/ + + /** + * Returns a hash code that is always the same for two Cells having a matching equals(..) result. + * Currently does not guard against nulls, but it could if necessary. + */ + public static int hashCode(Cell cell){ + if (cell == null) {// return 0 for empty Cell + return 0; + } + + //pre-calculate the 3 hashes made of byte ranges + int rowHash = Bytes.hashCode(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); + int familyHash = Bytes.hashCode(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + int qualifierHash = Bytes.hashCode(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + + //combine the 6 sub-hashes + int hash = 31 * rowHash + familyHash; + hash = 31 * hash + qualifierHash; + hash = 31 * hash + (int)cell.getTimestamp(); + hash = 31 * hash + cell.getTypeByte(); + hash = 31 * hash + (int)cell.getMvccVersion(); + return hash; + } + + + /******************** lengths *************************/ + + public static boolean areKeyLengthsEqual(Cell a, Cell b) { + return a.getRowLength() == b.getRowLength() + && a.getFamilyLength() == b.getFamilyLength() + && a.getQualifierLength() == b.getQualifierLength(); + } + + public static boolean areRowLengthsEqual(Cell a, Cell b) { + return a.getRowLength() == b.getRowLength(); + } + + + /***************** special cases ****************************/ + + /** + * special case for KeyValue.equals + */ + private static int compareStaticIgnoreMvccVersion(Cell a, Cell b) { + //row + int c = Bytes.compareTo( + a.getRowArray(), a.getRowOffset(), a.getRowLength(), + b.getRowArray(), b.getRowOffset(), b.getRowLength()); + if (c != 0) return c; + + //family + c = Bytes.compareTo( + a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(), + b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength()); + if (c != 0) return c; + + //qualifier + c = Bytes.compareTo( + a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(), + b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength()); + if (c != 0) return c; + + //timestamp: later sorts first + c = -Longs.compare(a.getTimestamp(), b.getTimestamp()); + if (c != 0) return c; + + //type + c = (0xff & a.getTypeByte()) - (0xff & b.getTypeByte()); + return c; + } + + /** + * special case for KeyValue.equals + */ + public static boolean equalsIgnoreMvccVersion(Cell a, Cell b){ + return 0 == compareStaticIgnoreMvccVersion(a, b); + } + +} diff --git a/hbase-common/src/main/java/org/apache/hbase/CellScanner.java b/hbase-common/src/main/java/org/apache/hbase/CellScanner.java new file mode 100644 index 0000000..1da8849 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hbase/CellScanner.java @@ -0,0 +1,68 @@ +/** + * 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.hbase; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hbase.Cell; + +/** + * An interface for iterating through a sequence of cells. Similar to Java's Iterator, but without + * the hasNext() or remove() methods. The hasNext() method is problematic because it may require + * actually loading the next object, which in turn requires storing the previous object somewhere. + * + *

    The core data block decoder should be as fast as possible, so we push the complexity and + * performance expense of concurrently tracking multiple cells to layers above the CellScanner. + *

    + * The {@link #get()} method will return a reference to a Cell implementation. This reference may + * or may not point to a reusable cell implementation, so users of the CellScanner should not, for + * example, accumulate a List of Cells. All of the references may point to the same object, which + * would be the latest state of the underlying Cell. In short, the Cell is mutable. Use + * {@link #getDeepCopy()} if you want a Cell that does not hold references (if the data is + * encoded or compressed, the call to {@link #getDeepCopy()} will cost more than a {@link #get()}. + *

    + * Typical usage: + * + *

    + * while (scanner.next()) {
    + *   Cell cell = scanner.get();
    + *   // do something
    + * }
    + * 
    + *

    Often used reading {@link Cell}s written by {@Link org.apache.hbase.io.CellOutputStream}. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface CellScanner { + /** + * @return the current Cell which may be mutable + */ + Cell get(); + + /** + * @return Get a deep copy of the current Cell or null if no current Cell + */ + Cell getDeepCopy(); + + /** + * Advance the scanner 1 cell. + * @return true if the next cell is found and {@link #get()} will return a valid Cell + */ + boolean next(); +} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hbase/CellTool.java b/hbase-common/src/main/java/org/apache/hbase/CellTool.java new file mode 100644 index 0000000..d05c813 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hbase/CellTool.java @@ -0,0 +1,123 @@ +/* + * 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.hbase; + +import java.nio.ByteBuffer; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.ByteRange; +import org.apache.hadoop.hbase.KeyValue; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public final class CellTool { + + /******************* ByteRange *******************************/ + + public static ByteRange fillRowRange(Cell cell, ByteRange range) { + return range.set(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); + } + + public static ByteRange fillFamilyRange(Cell cell, ByteRange range) { + return range.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + } + + public static ByteRange fillQualifierRange(Cell cell, ByteRange range) { + return range.set(cell.getQualifierArray(), cell.getQualifierOffset(), + cell.getQualifierLength()); + } + + + /***************** get individual arrays for tests ************/ + + public static byte[] getRowArray(Cell cell){ + byte[] output = new byte[cell.getRowLength()]; + copyRowTo(cell, output, 0); + return output; + } + + public static byte[] getFamilyArray(Cell cell){ + byte[] output = new byte[cell.getFamilyLength()]; + copyFamilyTo(cell, output, 0); + return output; + } + + public static byte[] getQualifierArray(Cell cell){ + byte[] output = new byte[cell.getQualifierLength()]; + copyQualifierTo(cell, output, 0); + return output; + } + + public static byte[] getValueArray(Cell cell){ + byte[] output = new byte[cell.getValueLength()]; + copyValueTo(cell, output, 0); + return output; + } + + + /******************** copyTo **********************************/ + + public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) { + System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset, + cell.getRowLength()); + return destinationOffset + cell.getRowLength(); + } + + public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) { + System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination, destinationOffset, + cell.getFamilyLength()); + return destinationOffset + cell.getFamilyLength(); + } + + public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) { + System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination, + destinationOffset, cell.getQualifierLength()); + return destinationOffset + cell.getQualifierLength(); + } + + public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) { + System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset, + cell.getValueLength()); + return destinationOffset + cell.getValueLength(); + } + + + /********************* misc *************************************/ + + public static byte getRowByte(Cell cell, int index) { + return cell.getRowArray()[cell.getRowOffset() + index]; + } + + public static ByteBuffer getValueBufferShallowCopy(Cell cell) { + ByteBuffer buffer = ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(), + cell.getValueLength()); +// buffer.position(buffer.limit());//make it look as if value was appended + return buffer; + } + + public static Cell createCell(final byte [] row, final byte [] family, final byte [] qualifier, + final long timestamp, final byte type, final byte [] value) { + // I need a Cell Factory here. Using KeyValue for now. TODO. + // TODO: Make a new Cell implementation that just carries these + // byte arrays. + return new KeyValue(row, family, qualifier, timestamp, + KeyValue.Type.codeToType(type), value); + } +} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hbase/cell/CellComparator.java b/hbase-common/src/main/java/org/apache/hbase/cell/CellComparator.java deleted file mode 100644 index 19d6b54..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/cell/CellComparator.java +++ /dev/null @@ -1,178 +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.hbase.cell; - -import java.io.Serializable; -import java.util.Comparator; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; - -import com.google.common.primitives.Longs; - -/** - * Compare two traditional HBase cells. - * - * Note: This comparator is not valid for -ROOT- and .META. tables. - */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -public class CellComparator implements Comparator, Serializable{ - private static final long serialVersionUID = -8760041766259623329L; - - @Override - public int compare(Cell a, Cell b) { - return compareStatic(a, b); - } - - - public static int compareStatic(Cell a, Cell b) { - //row - int c = Bytes.compareTo( - a.getRowArray(), a.getRowOffset(), a.getRowLength(), - b.getRowArray(), b.getRowOffset(), b.getRowLength()); - if (c != 0) return c; - - //family - c = Bytes.compareTo( - a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(), - b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength()); - if (c != 0) return c; - - //qualifier - c = Bytes.compareTo( - a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(), - b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength()); - if (c != 0) return c; - - //timestamp: later sorts first - c = -Longs.compare(a.getTimestamp(), b.getTimestamp()); - if (c != 0) return c; - - //type - c = (0xff & a.getTypeByte()) - (0xff & b.getTypeByte()); - if (c != 0) return c; - - //mvccVersion: later sorts first - return -Longs.compare(a.getMvccVersion(), b.getMvccVersion()); - } - - - /**************** equals ****************************/ - - public static boolean equals(Cell a, Cell b){ - if (!areKeyLengthsEqual(a, b)) { - return false; - } - //TODO compare byte[]'s in reverse since later bytes more likely to differ - return 0 == compareStatic(a, b); - } - - public static boolean equalsRow(Cell a, Cell b){ - if(!areRowLengthsEqual(a, b)){ - return false; - } - return 0 == Bytes.compareTo( - a.getRowArray(), a.getRowOffset(), a.getRowLength(), - b.getRowArray(), b.getRowOffset(), b.getRowLength()); - } - - - /********************* hashCode ************************/ - - /** - * Returns a hash code that is always the same for two Cells having a matching equals(..) result. - * Currently does not guard against nulls, but it could if necessary. - */ - public static int hashCode(Cell cell){ - if (cell == null) {// return 0 for empty Cell - return 0; - } - - //pre-calculate the 3 hashes made of byte ranges - int rowHash = Bytes.hashCode(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); - int familyHash = Bytes.hashCode(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); - int qualifierHash = Bytes.hashCode(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); - - //combine the 6 sub-hashes - int hash = 31 * rowHash + familyHash; - hash = 31 * hash + qualifierHash; - hash = 31 * hash + (int)cell.getTimestamp(); - hash = 31 * hash + cell.getTypeByte(); - hash = 31 * hash + (int)cell.getMvccVersion(); - return hash; - } - - - /******************** lengths *************************/ - - public static boolean areKeyLengthsEqual(Cell a, Cell b) { - return a.getRowLength() == b.getRowLength() - && a.getFamilyLength() == b.getFamilyLength() - && a.getQualifierLength() == b.getQualifierLength(); - } - - public static boolean areRowLengthsEqual(Cell a, Cell b) { - return a.getRowLength() == b.getRowLength(); - } - - - /***************** special cases ****************************/ - - /** - * special case for KeyValue.equals - */ - private static int compareStaticIgnoreMvccVersion(Cell a, Cell b) { - //row - int c = Bytes.compareTo( - a.getRowArray(), a.getRowOffset(), a.getRowLength(), - b.getRowArray(), b.getRowOffset(), b.getRowLength()); - if (c != 0) return c; - - //family - c = Bytes.compareTo( - a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(), - b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength()); - if (c != 0) return c; - - //qualifier - c = Bytes.compareTo( - a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(), - b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength()); - if (c != 0) return c; - - //timestamp: later sorts first - c = -Longs.compare(a.getTimestamp(), b.getTimestamp()); - if (c != 0) return c; - - //type - c = (0xff & a.getTypeByte()) - (0xff & b.getTypeByte()); - return c; - } - - /** - * special case for KeyValue.equals - */ - public static boolean equalsIgnoreMvccVersion(Cell a, Cell b){ - return 0 == compareStaticIgnoreMvccVersion(a, b); - } - -} diff --git a/hbase-common/src/main/java/org/apache/hbase/cell/CellOutputStream.java b/hbase-common/src/main/java/org/apache/hbase/cell/CellOutputStream.java deleted file mode 100644 index fcaf27e..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/cell/CellOutputStream.java +++ /dev/null @@ -1,50 +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.hbase.cell; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hbase.Cell; - -/** - * Accepts a stream of Cells and adds them to its internal data structure. This can be used to build - * a block of cells during compactions and flushes, or to build a byte[] to send to the client. This - * could be backed by a List, but more efficient implementations will append results to a - * byte[] to eliminate overhead, and possibly encode the cells further. - */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -public interface CellOutputStream { - - /** - * Implementation must copy the entire state of the Cell. If the appended Cell is modified - * immediately after the append method returns, the modifications must have absolutely no effect - * on the copy of the Cell that was added to the appender. For example, calling someList.add(cell) - * is not correct. - */ - void write(Cell cell); - - /** - * Let the implementation decide what to do. Usually means writing accumulated data into a byte[] - * that can then be read from the implementation to be sent to disk, put in the block cache, or - * sent over the network. - */ - void flush(); - -} diff --git a/hbase-common/src/main/java/org/apache/hbase/cell/CellScannerPosition.java b/hbase-common/src/main/java/org/apache/hbase/cell/CellScannerPosition.java deleted file mode 100644 index eeadf5f..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/cell/CellScannerPosition.java +++ /dev/null @@ -1,68 +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.hbase.cell; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * An indicator of the state of the scanner after an operation such as nextCell() or positionAt(..). - * For example: - *

    - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public enum CellScannerPosition { - - /** - * getCurrentCell() will NOT return a valid cell. Calling nextCell() will advance to the first - * cell. - */ - BEFORE_FIRST, - - /** - * getCurrentCell() will return a valid cell, but it is not the cell requested by positionAt(..), - * rather it is the nearest cell before the requested cell. - */ - BEFORE, - - /** - * getCurrentCell() will return a valid cell, and it is exactly the cell that was requested by - * positionAt(..). - */ - AT, - - /** - * getCurrentCell() will return a valid cell, but it is not the cell requested by positionAt(..), - * rather it is the nearest cell after the requested cell. - */ - AFTER, - - /** - * getCurrentCell() will NOT return a valid cell. Calling nextCell() will have no effect. - */ - AFTER_LAST - -} diff --git a/hbase-common/src/main/java/org/apache/hbase/cell/CellTool.java b/hbase-common/src/main/java/org/apache/hbase/cell/CellTool.java deleted file mode 100644 index 229ca36..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/cell/CellTool.java +++ /dev/null @@ -1,118 +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.hbase.cell; - -import java.nio.ByteBuffer; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.hbase.util.ByteRange; -import org.apache.hbase.Cell; - -@InterfaceAudience.Private -@InterfaceStability.Evolving -public final class CellTool { - - /******************* ByteRange *******************************/ - - public static ByteRange fillRowRange(Cell cell, ByteRange range) { - return range.set(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); - } - - public static ByteRange fillFamilyRange(Cell cell, ByteRange range) { - return range.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); - } - - public static ByteRange fillQualifierRange(Cell cell, ByteRange range) { - return range.set(cell.getQualifierArray(), cell.getQualifierOffset(), - cell.getQualifierLength()); - } - - - /***************** get individual arrays for tests ************/ - - public static byte[] getRowArray(Cell cell){ - byte[] output = new byte[cell.getRowLength()]; - copyRowTo(cell, output, 0); - return output; - } - - public static byte[] getFamilyArray(Cell cell){ - byte[] output = new byte[cell.getFamilyLength()]; - copyFamilyTo(cell, output, 0); - return output; - } - - public static byte[] getQualifierArray(Cell cell){ - byte[] output = new byte[cell.getQualifierLength()]; - copyQualifierTo(cell, output, 0); - return output; - } - - public static byte[] getValueArray(Cell cell){ - byte[] output = new byte[cell.getValueLength()]; - copyValueTo(cell, output, 0); - return output; - } - - - /******************** copyTo **********************************/ - - public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) { - System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset, - cell.getRowLength()); - return destinationOffset + cell.getRowLength(); - } - - public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) { - System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination, destinationOffset, - cell.getFamilyLength()); - return destinationOffset + cell.getFamilyLength(); - } - - public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) { - System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination, - destinationOffset, cell.getQualifierLength()); - return destinationOffset + cell.getQualifierLength(); - } - - public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) { - System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset, - cell.getValueLength()); - return destinationOffset + cell.getValueLength(); - } - - - /********************* misc *************************************/ - - public static byte getRowByte(Cell cell, int index) { - return cell.getRowArray()[cell.getRowOffset() + index]; - } - - - /********************** KeyValue (move to KeyValueUtils) *********************/ - - public static ByteBuffer getValueBufferShallowCopy(Cell cell) { - ByteBuffer buffer = ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(), - cell.getValueLength()); -// buffer.position(buffer.limit());//make it look as if value was appended - return buffer; - } - -} diff --git a/hbase-common/src/main/java/org/apache/hbase/codec/CodecException.java b/hbase-common/src/main/java/org/apache/hbase/codec/CodecException.java new file mode 100644 index 0000000..352e1a6 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hbase/codec/CodecException.java @@ -0,0 +1,39 @@ +/** + * 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.hbase.codec; + +import java.io.IOException; + +public class CodecException extends IOException { + private static final long serialVersionUID = -2850095011686914405L; + + public CodecException() { + } + + public CodecException(String message) { + super(message); + } + + public CodecException(Throwable t) { + super(t); + } + + public CodecException(String message, Throwable t) { + super(message, t); + } +} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hbase/io/CellOutputStream.java b/hbase-common/src/main/java/org/apache/hbase/io/CellOutputStream.java new file mode 100644 index 0000000..f832e8d --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hbase/io/CellOutputStream.java @@ -0,0 +1,55 @@ +/* + * 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.hbase.io; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hbase.Cell; +import org.apache.hbase.CellScanner; + +/** + * Accepts a stream of Cells. This can be used to build a block of cells during compactions + * and flushes, or to build a byte[] to send to the client. This could be backed by a + * List, but more efficient implementations will append results to a + * byte[] to eliminate overhead, and possibly encode the cells further. + *

    To read Cells, use {@link CellScanner} + * @see CellScanner + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface CellOutputStream { + /** + * Implementation must copy the entire state of the Cell. If the written Cell is modified + * immediately after the write method returns, the modifications must have absolutely no effect + * on the copy of the Cell that was added in the write. + * @param cell Cell to write out + * @throws IOException + */ + void write(Cell cell) throws IOException; + + /** + * Let the implementation decide what to do. Usually means writing accumulated data into a byte[] + * that can then be read from the implementation to be sent to disk, put in the block cache, or + * sent over the network. + * @throws IOException + */ + void flush() throws IOException; +} \ No newline at end of file diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml index 38be369..181641b 100644 --- a/hbase-protocol/pom.xml +++ b/hbase-protocol/pom.xml @@ -53,11 +53,16 @@ - - - com.google.protobuf - protobuf-java - + + + org.apache.hbase + hbase-common + + + + com.google.protobuf + protobuf-java + @@ -74,4 +79,4 @@ - \ No newline at end of file + diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java index 5aff852..97b5ef8 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java @@ -2048,14 +2048,14 @@ public final class ClientProtos { public interface ResultOrBuilder extends com.google.protobuf.MessageOrBuilder { - // repeated .KeyValue keyValue = 1; - java.util.List + // repeated .Cell keyValue = 1; + java.util.List getKeyValueList(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue getKeyValue(int index); + org.apache.hbase.protobuf.generated.CellProtos.Cell getKeyValue(int index); int getKeyValueCount(); - java.util.List + java.util.List getKeyValueOrBuilderList(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder getKeyValueOrBuilder( + org.apache.hbase.protobuf.generated.CellProtos.CellOrBuilder getKeyValueOrBuilder( int index); } public static final class Result extends @@ -2086,23 +2086,23 @@ public final class ClientProtos { return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Result_fieldAccessorTable; } - // repeated .KeyValue keyValue = 1; + // repeated .Cell keyValue = 1; public static final int KEYVALUE_FIELD_NUMBER = 1; - private java.util.List keyValue_; - public java.util.List getKeyValueList() { + private java.util.List keyValue_; + public java.util.List getKeyValueList() { return keyValue_; } - public java.util.List + public java.util.List getKeyValueOrBuilderList() { return keyValue_; } public int getKeyValueCount() { return keyValue_.size(); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue getKeyValue(int index) { + public org.apache.hbase.protobuf.generated.CellProtos.Cell getKeyValue(int index) { return keyValue_.get(index); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder getKeyValueOrBuilder( + public org.apache.hbase.protobuf.generated.CellProtos.CellOrBuilder getKeyValueOrBuilder( int index) { return keyValue_.get(index); } @@ -2115,12 +2115,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - for (int i = 0; i < getKeyValueCount(); i++) { - if (!getKeyValue(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -2397,12 +2391,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - for (int i = 0; i < getKeyValueCount(); i++) { - if (!getKeyValue(i).isInitialized()) { - - return false; - } - } return true; } @@ -2430,7 +2418,7 @@ public final class ClientProtos { break; } case 10: { - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.newBuilder(); + org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder subBuilder = org.apache.hbase.protobuf.generated.CellProtos.Cell.newBuilder(); input.readMessage(subBuilder, extensionRegistry); addKeyValue(subBuilder.buildPartial()); break; @@ -2441,20 +2429,20 @@ public final class ClientProtos { private int bitField0_; - // repeated .KeyValue keyValue = 1; - private java.util.List keyValue_ = + // repeated .Cell keyValue = 1; + private java.util.List keyValue_ = java.util.Collections.emptyList(); private void ensureKeyValueIsMutable() { if (!((bitField0_ & 0x00000001) == 0x00000001)) { - keyValue_ = new java.util.ArrayList(keyValue_); + keyValue_ = new java.util.ArrayList(keyValue_); bitField0_ |= 0x00000001; } } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder> keyValueBuilder_; + org.apache.hbase.protobuf.generated.CellProtos.Cell, org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder, org.apache.hbase.protobuf.generated.CellProtos.CellOrBuilder> keyValueBuilder_; - public java.util.List getKeyValueList() { + public java.util.List getKeyValueList() { if (keyValueBuilder_ == null) { return java.util.Collections.unmodifiableList(keyValue_); } else { @@ -2468,7 +2456,7 @@ public final class ClientProtos { return keyValueBuilder_.getCount(); } } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue getKeyValue(int index) { + public org.apache.hbase.protobuf.generated.CellProtos.Cell getKeyValue(int index) { if (keyValueBuilder_ == null) { return keyValue_.get(index); } else { @@ -2476,7 +2464,7 @@ public final class ClientProtos { } } public Builder setKeyValue( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue value) { + int index, org.apache.hbase.protobuf.generated.CellProtos.Cell value) { if (keyValueBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2490,7 +2478,7 @@ public final class ClientProtos { return this; } public Builder setKeyValue( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder builderForValue) { + int index, org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder builderForValue) { if (keyValueBuilder_ == null) { ensureKeyValueIsMutable(); keyValue_.set(index, builderForValue.build()); @@ -2500,7 +2488,7 @@ public final class ClientProtos { } return this; } - public Builder addKeyValue(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue value) { + public Builder addKeyValue(org.apache.hbase.protobuf.generated.CellProtos.Cell value) { if (keyValueBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2514,7 +2502,7 @@ public final class ClientProtos { return this; } public Builder addKeyValue( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue value) { + int index, org.apache.hbase.protobuf.generated.CellProtos.Cell value) { if (keyValueBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2528,7 +2516,7 @@ public final class ClientProtos { return this; } public Builder addKeyValue( - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder builderForValue) { + org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder builderForValue) { if (keyValueBuilder_ == null) { ensureKeyValueIsMutable(); keyValue_.add(builderForValue.build()); @@ -2539,7 +2527,7 @@ public final class ClientProtos { return this; } public Builder addKeyValue( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder builderForValue) { + int index, org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder builderForValue) { if (keyValueBuilder_ == null) { ensureKeyValueIsMutable(); keyValue_.add(index, builderForValue.build()); @@ -2550,7 +2538,7 @@ public final class ClientProtos { return this; } public Builder addAllKeyValue( - java.lang.Iterable values) { + java.lang.Iterable values) { if (keyValueBuilder_ == null) { ensureKeyValueIsMutable(); super.addAll(values, keyValue_); @@ -2580,18 +2568,18 @@ public final class ClientProtos { } return this; } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder getKeyValueBuilder( + public org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder getKeyValueBuilder( int index) { return getKeyValueFieldBuilder().getBuilder(index); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder getKeyValueOrBuilder( + public org.apache.hbase.protobuf.generated.CellProtos.CellOrBuilder getKeyValueOrBuilder( int index) { if (keyValueBuilder_ == null) { return keyValue_.get(index); } else { return keyValueBuilder_.getMessageOrBuilder(index); } } - public java.util.List + public java.util.List getKeyValueOrBuilderList() { if (keyValueBuilder_ != null) { return keyValueBuilder_.getMessageOrBuilderList(); @@ -2599,25 +2587,25 @@ public final class ClientProtos { return java.util.Collections.unmodifiableList(keyValue_); } } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder addKeyValueBuilder() { + public org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder addKeyValueBuilder() { return getKeyValueFieldBuilder().addBuilder( - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.getDefaultInstance()); + org.apache.hbase.protobuf.generated.CellProtos.Cell.getDefaultInstance()); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder addKeyValueBuilder( + public org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder addKeyValueBuilder( int index) { return getKeyValueFieldBuilder().addBuilder( - index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.getDefaultInstance()); + index, org.apache.hbase.protobuf.generated.CellProtos.Cell.getDefaultInstance()); } - public java.util.List + public java.util.List getKeyValueBuilderList() { return getKeyValueFieldBuilder().getBuilderList(); } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder> + org.apache.hbase.protobuf.generated.CellProtos.Cell, org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder, org.apache.hbase.protobuf.generated.CellProtos.CellOrBuilder> getKeyValueFieldBuilder() { if (keyValueBuilder_ == null) { keyValueBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder>( + org.apache.hbase.protobuf.generated.CellProtos.Cell, org.apache.hbase.protobuf.generated.CellProtos.Cell.Builder, org.apache.hbase.protobuf.generated.CellProtos.CellOrBuilder>( keyValue_, ((bitField0_ & 0x00000001) == 0x00000001), getParentForChildren(), @@ -4380,12 +4368,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (hasResult()) { - if (!getResult().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -4668,12 +4650,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - if (hasResult()) { - if (!getResult().isInitialized()) { - - return false; - } - } return true; } @@ -4932,12 +4908,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -5246,12 +5216,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { - - return false; - } - } return true; } @@ -9934,12 +9898,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (hasResult()) { - if (!getResult().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -10222,12 +10180,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - if (hasResult()) { - if (!getResult().isInitialized()) { - - return false; - } - } return true; } @@ -13227,12 +13179,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -13611,12 +13557,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { - - return false; - } - } return true; } @@ -18331,12 +18271,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (hasValue()) { - if (!getValue().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } if (hasException()) { if (!getException().isInitialized()) { memoizedIsInitialized = 0; @@ -18634,12 +18568,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - if (hasValue()) { - if (!getValue().isInitialized()) { - - return false; - } - } if (hasException()) { if (!getException().isInitialized()) { @@ -21060,91 +20988,92 @@ public final class ClientProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\014Client.proto\032\013hbase.proto\032\020Comparator." + - "proto\"+\n\006Column\022\016\n\006family\030\001 \002(\014\022\021\n\tquali" + - "fier\030\002 \003(\014\"\342\001\n\003Get\022\013\n\003row\030\001 \002(\014\022\027\n\006colum" + - "n\030\002 \003(\0132\007.Column\022!\n\tattribute\030\003 \003(\0132\016.Na" + - "meBytesPair\022\027\n\006filter\030\004 \001(\0132\007.Filter\022\035\n\t" + - "timeRange\030\005 \001(\0132\n.TimeRange\022\026\n\013maxVersio" + - "ns\030\006 \001(\r:\0011\022\031\n\013cacheBlocks\030\007 \001(\010:\004true\022\022" + - "\n\nstoreLimit\030\010 \001(\r\022\023\n\013storeOffset\030\t \001(\r\"" + - "%\n\006Result\022\033\n\010keyValue\030\001 \003(\0132\t.KeyValue\"r" + - "\n\nGetRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpe", - "cifier\022\021\n\003get\030\002 \002(\0132\004.Get\022\030\n\020closestRowB" + - "efore\030\003 \001(\010\022\025\n\rexistenceOnly\030\004 \001(\010\"w\n\017Mu" + - "ltiGetRequest\022 \n\006region\030\001 \002(\0132\020.RegionSp" + - "ecifier\022\021\n\003get\030\002 \003(\0132\004.Get\022\030\n\020closestRow" + - "Before\030\003 \001(\010\022\025\n\rexistenceOnly\030\004 \001(\010\"6\n\013G" + - "etResponse\022\027\n\006result\030\001 \001(\0132\007.Result\022\016\n\006e" + - "xists\030\002 \001(\010\";\n\020MultiGetResponse\022\027\n\006resul" + - "t\030\001 \003(\0132\007.Result\022\016\n\006exists\030\002 \003(\010\"\177\n\tCond" + - "ition\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002 \002(\014\022\021\n\tqu" + - "alifier\030\003 \002(\014\022!\n\013compareType\030\004 \002(\0162\014.Com", - "pareType\022\037\n\ncomparator\030\005 \002(\0132\013.Comparato" + - "r\"\266\004\n\006Mutate\022\013\n\003row\030\001 \002(\014\022&\n\nmutateType\030" + - "\002 \002(\0162\022.Mutate.MutateType\022(\n\013columnValue" + - "\030\003 \003(\0132\023.Mutate.ColumnValue\022!\n\tattribute" + - "\030\004 \003(\0132\016.NameBytesPair\022\021\n\ttimestamp\030\005 \001(" + - "\004\022\030\n\nwriteToWAL\030\006 \001(\010:\004true\022\035\n\ttimeRange" + - "\030\n \001(\0132\n.TimeRange\032\310\001\n\013ColumnValue\022\016\n\006fa" + - "mily\030\001 \002(\014\022:\n\016qualifierValue\030\002 \003(\0132\".Mut" + - "ate.ColumnValue.QualifierValue\032m\n\016Qualif" + - "ierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001", - "(\014\022\021\n\ttimestamp\030\003 \001(\004\022&\n\ndeleteType\030\004 \001(" + - "\0162\022.Mutate.DeleteType\"<\n\nMutateType\022\n\n\006A" + - "PPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELET" + - "E\020\003\"U\n\nDeleteType\022\026\n\022DELETE_ONE_VERSION\020" + - "\000\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001\022\021\n\rDELET" + - "E_FAMILY\020\002\"i\n\rMutateRequest\022 \n\006region\030\001 " + - "\002(\0132\020.RegionSpecifier\022\027\n\006mutate\030\002 \002(\0132\007." + - "Mutate\022\035\n\tcondition\030\003 \001(\0132\n.Condition\"<\n" + - "\016MutateResponse\022\027\n\006result\030\001 \001(\0132\007.Result" + - "\022\021\n\tprocessed\030\002 \001(\010\"\307\002\n\004Scan\022\027\n\006column\030\001", - " \003(\0132\007.Column\022!\n\tattribute\030\002 \003(\0132\016.NameB" + - "ytesPair\022\020\n\010startRow\030\003 \001(\014\022\017\n\007stopRow\030\004 " + - "\001(\014\022\027\n\006filter\030\005 \001(\0132\007.Filter\022\035\n\ttimeRang" + - "e\030\006 \001(\0132\n.TimeRange\022\026\n\013maxVersions\030\007 \001(\r" + - ":\0011\022\031\n\013cacheBlocks\030\010 \001(\010:\004true\022\021\n\tbatchS" + - "ize\030\t \001(\r\022\025\n\rmaxResultSize\030\n \001(\004\022\022\n\nstor" + - "eLimit\030\013 \001(\r\022\023\n\013storeOffset\030\014 \001(\r\022\"\n\032loa" + - "dColumnFamiliesOnDemand\030\r \001(\010\"\230\001\n\013ScanRe" + - "quest\022 \n\006region\030\001 \001(\0132\020.RegionSpecifier\022" + - "\023\n\004scan\030\002 \001(\0132\005.Scan\022\021\n\tscannerId\030\003 \001(\004\022", - "\024\n\014numberOfRows\030\004 \001(\r\022\024\n\014closeScanner\030\005 " + - "\001(\010\022\023\n\013nextCallSeq\030\006 \001(\004\"u\n\014ScanResponse" + - "\022\027\n\006result\030\001 \003(\0132\007.Result\022\021\n\tscannerId\030\002" + - " \001(\004\022\023\n\013moreResults\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022\027" + - "\n\017resultSizeBytes\030\005 \001(\004\"\260\001\n\024BulkLoadHFil" + - "eRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifi" + - "er\0224\n\nfamilyPath\030\002 \003(\0132 .BulkLoadHFileRe" + - "quest.FamilyPath\022\024\n\014assignSeqNum\030\003 \001(\010\032*" + - "\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002" + - "(\t\"\'\n\025BulkLoadHFileResponse\022\016\n\006loaded\030\001 ", - "\002(\010\"_\n\026CoprocessorServiceCall\022\013\n\003row\030\001 \002" + - "(\014\022\023\n\013serviceName\030\002 \002(\t\022\022\n\nmethodName\030\003 " + - "\002(\t\022\017\n\007request\030\004 \002(\014\"d\n\031CoprocessorServi" + - "ceRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecif" + - "ier\022%\n\004call\030\002 \002(\0132\027.CoprocessorServiceCa" + - "ll\"]\n\032CoprocessorServiceResponse\022 \n\006regi" + - "on\030\001 \002(\0132\020.RegionSpecifier\022\035\n\005value\030\002 \002(" + - "\0132\016.NameBytesPair\"9\n\013MultiAction\022\027\n\006muta" + - "te\030\001 \001(\0132\007.Mutate\022\021\n\003get\030\002 \001(\0132\004.Get\"I\n\014" + - "ActionResult\022\026\n\005value\030\001 \001(\0132\007.Result\022!\n\t", - "exception\030\002 \001(\0132\016.NameBytesPair\"^\n\014Multi" + - "Request\022 \n\006region\030\001 \002(\0132\020.RegionSpecifie" + - "r\022\034\n\006action\030\002 \003(\0132\014.MultiAction\022\016\n\006atomi" + - "c\030\003 \001(\010\".\n\rMultiResponse\022\035\n\006result\030\001 \003(\013" + - "2\r.ActionResult2\342\002\n\rClientService\022 \n\003get" + - "\022\013.GetRequest\032\014.GetResponse\022/\n\010multiGet\022" + - "\020.MultiGetRequest\032\021.MultiGetResponse\022)\n\006" + - "mutate\022\016.MutateRequest\032\017.MutateResponse\022" + - "#\n\004scan\022\014.ScanRequest\032\r.ScanResponse\022>\n\r" + - "bulkLoadHFile\022\025.BulkLoadHFileRequest\032\026.B", - "ulkLoadHFileResponse\022F\n\013execService\022\032.Co" + - "processorServiceRequest\032\033.CoprocessorSer" + - "viceResponse\022&\n\005multi\022\r.MultiRequest\032\016.M" + - "ultiResponseBB\n*org.apache.hadoop.hbase." + - "protobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001" + "\n\014Client.proto\032\nCell.proto\032\013hbase.proto\032" + + "\020Comparator.proto\"+\n\006Column\022\016\n\006family\030\001 " + + "\002(\014\022\021\n\tqualifier\030\002 \003(\014\"\342\001\n\003Get\022\013\n\003row\030\001 " + + "\002(\014\022\027\n\006column\030\002 \003(\0132\007.Column\022!\n\tattribut" + + "e\030\003 \003(\0132\016.NameBytesPair\022\027\n\006filter\030\004 \001(\0132" + + "\007.Filter\022\035\n\ttimeRange\030\005 \001(\0132\n.TimeRange\022" + + "\026\n\013maxVersions\030\006 \001(\r:\0011\022\031\n\013cacheBlocks\030\007" + + " \001(\010:\004true\022\022\n\nstoreLimit\030\010 \001(\r\022\023\n\013storeO" + + "ffset\030\t \001(\r\"!\n\006Result\022\027\n\010keyValue\030\001 \003(\0132" + + "\005.Cell\"r\n\nGetRequest\022 \n\006region\030\001 \002(\0132\020.R", + "egionSpecifier\022\021\n\003get\030\002 \002(\0132\004.Get\022\030\n\020clo" + + "sestRowBefore\030\003 \001(\010\022\025\n\rexistenceOnly\030\004 \001" + + "(\010\"w\n\017MultiGetRequest\022 \n\006region\030\001 \002(\0132\020." + + "RegionSpecifier\022\021\n\003get\030\002 \003(\0132\004.Get\022\030\n\020cl" + + "osestRowBefore\030\003 \001(\010\022\025\n\rexistenceOnly\030\004 " + + "\001(\010\"6\n\013GetResponse\022\027\n\006result\030\001 \001(\0132\007.Res" + + "ult\022\016\n\006exists\030\002 \001(\010\";\n\020MultiGetResponse\022" + + "\027\n\006result\030\001 \003(\0132\007.Result\022\016\n\006exists\030\002 \003(\010" + + "\"\177\n\tCondition\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002 \002" + + "(\014\022\021\n\tqualifier\030\003 \002(\014\022!\n\013compareType\030\004 \002", + "(\0162\014.CompareType\022\037\n\ncomparator\030\005 \002(\0132\013.C" + + "omparator\"\266\004\n\006Mutate\022\013\n\003row\030\001 \002(\014\022&\n\nmut" + + "ateType\030\002 \002(\0162\022.Mutate.MutateType\022(\n\013col" + + "umnValue\030\003 \003(\0132\023.Mutate.ColumnValue\022!\n\ta" + + "ttribute\030\004 \003(\0132\016.NameBytesPair\022\021\n\ttimest" + + "amp\030\005 \001(\004\022\030\n\nwriteToWAL\030\006 \001(\010:\004true\022\035\n\tt" + + "imeRange\030\n \001(\0132\n.TimeRange\032\310\001\n\013ColumnVal" + + "ue\022\016\n\006family\030\001 \002(\014\022:\n\016qualifierValue\030\002 \003" + + "(\0132\".Mutate.ColumnValue.QualifierValue\032m" + + "\n\016QualifierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005v", + "alue\030\002 \001(\014\022\021\n\ttimestamp\030\003 \001(\004\022&\n\ndeleteT" + + "ype\030\004 \001(\0162\022.Mutate.DeleteType\"<\n\nMutateT" + + "ype\022\n\n\006APPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022" + + "\n\n\006DELETE\020\003\"U\n\nDeleteType\022\026\n\022DELETE_ONE_" + + "VERSION\020\000\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001\022" + + "\021\n\rDELETE_FAMILY\020\002\"i\n\rMutateRequest\022 \n\006r" + + "egion\030\001 \002(\0132\020.RegionSpecifier\022\027\n\006mutate\030" + + "\002 \002(\0132\007.Mutate\022\035\n\tcondition\030\003 \001(\0132\n.Cond" + + "ition\"<\n\016MutateResponse\022\027\n\006result\030\001 \001(\0132" + + "\007.Result\022\021\n\tprocessed\030\002 \001(\010\"\307\002\n\004Scan\022\027\n\006", + "column\030\001 \003(\0132\007.Column\022!\n\tattribute\030\002 \003(\013" + + "2\016.NameBytesPair\022\020\n\010startRow\030\003 \001(\014\022\017\n\007st" + + "opRow\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007.Filter\022\035\n\t" + + "timeRange\030\006 \001(\0132\n.TimeRange\022\026\n\013maxVersio" + + "ns\030\007 \001(\r:\0011\022\031\n\013cacheBlocks\030\010 \001(\010:\004true\022\021" + + "\n\tbatchSize\030\t \001(\r\022\025\n\rmaxResultSize\030\n \001(\004" + + "\022\022\n\nstoreLimit\030\013 \001(\r\022\023\n\013storeOffset\030\014 \001(" + + "\r\022\"\n\032loadColumnFamiliesOnDemand\030\r \001(\010\"\230\001" + + "\n\013ScanRequest\022 \n\006region\030\001 \001(\0132\020.RegionSp" + + "ecifier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\021\n\tscannerI", + "d\030\003 \001(\004\022\024\n\014numberOfRows\030\004 \001(\r\022\024\n\014closeSc" + + "anner\030\005 \001(\010\022\023\n\013nextCallSeq\030\006 \001(\004\"u\n\014Scan" + + "Response\022\027\n\006result\030\001 \003(\0132\007.Result\022\021\n\tsca" + + "nnerId\030\002 \001(\004\022\023\n\013moreResults\030\003 \001(\010\022\013\n\003ttl" + + "\030\004 \001(\r\022\027\n\017resultSizeBytes\030\005 \001(\004\"\260\001\n\024Bulk" + + "LoadHFileRequest\022 \n\006region\030\001 \002(\0132\020.Regio" + + "nSpecifier\0224\n\nfamilyPath\030\002 \003(\0132 .BulkLoa" + + "dHFileRequest.FamilyPath\022\024\n\014assignSeqNum" + + "\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004" + + "path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022\016\n\006l", + "oaded\030\001 \002(\010\"_\n\026CoprocessorServiceCall\022\013\n" + + "\003row\030\001 \002(\014\022\023\n\013serviceName\030\002 \002(\t\022\022\n\nmetho" + + "dName\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"d\n\031Coproces" + + "sorServiceRequest\022 \n\006region\030\001 \002(\0132\020.Regi" + + "onSpecifier\022%\n\004call\030\002 \002(\0132\027.CoprocessorS" + + "erviceCall\"]\n\032CoprocessorServiceResponse" + + "\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\035\n\005va" + + "lue\030\002 \002(\0132\016.NameBytesPair\"9\n\013MultiAction" + + "\022\027\n\006mutate\030\001 \001(\0132\007.Mutate\022\021\n\003get\030\002 \001(\0132\004" + + ".Get\"I\n\014ActionResult\022\026\n\005value\030\001 \001(\0132\007.Re", + "sult\022!\n\texception\030\002 \001(\0132\016.NameBytesPair\"" + + "^\n\014MultiRequest\022 \n\006region\030\001 \002(\0132\020.Region" + + "Specifier\022\034\n\006action\030\002 \003(\0132\014.MultiAction\022" + + "\016\n\006atomic\030\003 \001(\010\".\n\rMultiResponse\022\035\n\006resu" + + "lt\030\001 \003(\0132\r.ActionResult2\342\002\n\rClientServic" + + "e\022 \n\003get\022\013.GetRequest\032\014.GetResponse\022/\n\010m" + + "ultiGet\022\020.MultiGetRequest\032\021.MultiGetResp" + + "onse\022)\n\006mutate\022\016.MutateRequest\032\017.MutateR" + + "esponse\022#\n\004scan\022\014.ScanRequest\032\r.ScanResp" + + "onse\022>\n\rbulkLoadHFile\022\025.BulkLoadHFileReq", + "uest\032\026.BulkLoadHFileResponse\022F\n\013execServ" + + "ice\022\032.CoprocessorServiceRequest\032\033.Coproc" + + "essorServiceResponse\022&\n\005multi\022\r.MultiReq" + + "uest\032\016.MultiResponseBB\n*org.apache.hadoo" + + "p.hbase.protobuf.generatedB\014ClientProtos" + + "H\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -21365,6 +21294,7 @@ public final class ClientProtos { com.google.protobuf.Descriptors.FileDescriptor .internalBuildGeneratedFileFrom(descriptorData, new com.google.protobuf.Descriptors.FileDescriptor[] { + org.apache.hbase.protobuf.generated.CellProtos.getDescriptor(), org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(), org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.getDescriptor(), }, assigner); diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java index a282e86..b1888f5 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java @@ -92,87 +92,6 @@ public final class HBaseProtos { // @@protoc_insertion_point(enum_scope:CompareType) } - public enum KeyType - implements com.google.protobuf.ProtocolMessageEnum { - MINIMUM(0, 0), - PUT(1, 4), - DELETE(2, 8), - DELETE_COLUMN(3, 12), - DELETE_FAMILY(4, 14), - MAXIMUM(5, 255), - ; - - public static final int MINIMUM_VALUE = 0; - public static final int PUT_VALUE = 4; - public static final int DELETE_VALUE = 8; - public static final int DELETE_COLUMN_VALUE = 12; - public static final int DELETE_FAMILY_VALUE = 14; - public static final int MAXIMUM_VALUE = 255; - - - public final int getNumber() { return value; } - - public static KeyType valueOf(int value) { - switch (value) { - case 0: return MINIMUM; - case 4: return PUT; - case 8: return DELETE; - case 12: return DELETE_COLUMN; - case 14: return DELETE_FAMILY; - case 255: return MAXIMUM; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public KeyType findValueByNumber(int number) { - return KeyType.valueOf(number); - } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor().getEnumTypes().get(1); - } - - private static final KeyType[] VALUES = { - MINIMUM, PUT, DELETE, DELETE_COLUMN, DELETE_FAMILY, MAXIMUM, - }; - - public static KeyType valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - return VALUES[desc.getIndex()]; - } - - private final int index; - private final int value; - - private KeyType(int index, int value) { - this.index = index; - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:KeyType) - } - public interface TableSchemaOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -7331,9 +7250,9 @@ public final class HBaseProtos { boolean hasTimestamp(); long getTimestamp(); - // optional .KeyType keyType = 5; + // optional .CellType keyType = 5; boolean hasKeyType(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType getKeyType(); + org.apache.hbase.protobuf.generated.CellProtos.CellType getKeyType(); // optional bytes value = 6; boolean hasValue(); @@ -7408,13 +7327,13 @@ public final class HBaseProtos { return timestamp_; } - // optional .KeyType keyType = 5; + // optional .CellType keyType = 5; public static final int KEYTYPE_FIELD_NUMBER = 5; - private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType keyType_; + private org.apache.hbase.protobuf.generated.CellProtos.CellType keyType_; public boolean hasKeyType() { return ((bitField0_ & 0x00000010) == 0x00000010); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType getKeyType() { + public org.apache.hbase.protobuf.generated.CellProtos.CellType getKeyType() { return keyType_; } @@ -7433,7 +7352,7 @@ public final class HBaseProtos { family_ = com.google.protobuf.ByteString.EMPTY; qualifier_ = com.google.protobuf.ByteString.EMPTY; timestamp_ = 0L; - keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM; + keyType_ = org.apache.hbase.protobuf.generated.CellProtos.CellType.MINIMUM; value_ = com.google.protobuf.ByteString.EMPTY; } private byte memoizedIsInitialized = -1; @@ -7721,7 +7640,7 @@ public final class HBaseProtos { bitField0_ = (bitField0_ & ~0x00000004); timestamp_ = 0L; bitField0_ = (bitField0_ & ~0x00000008); - keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM; + keyType_ = org.apache.hbase.protobuf.generated.CellProtos.CellType.MINIMUM; bitField0_ = (bitField0_ & ~0x00000010); value_ = com.google.protobuf.ByteString.EMPTY; bitField0_ = (bitField0_ & ~0x00000020); @@ -7886,7 +7805,7 @@ public final class HBaseProtos { } case 40: { int rawValue = input.readEnum(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.valueOf(rawValue); + org.apache.hbase.protobuf.generated.CellProtos.CellType value = org.apache.hbase.protobuf.generated.CellProtos.CellType.valueOf(rawValue); if (value == null) { unknownFields.mergeVarintField(5, rawValue); } else { @@ -7999,15 +7918,15 @@ public final class HBaseProtos { return this; } - // optional .KeyType keyType = 5; - private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM; + // optional .CellType keyType = 5; + private org.apache.hbase.protobuf.generated.CellProtos.CellType keyType_ = org.apache.hbase.protobuf.generated.CellProtos.CellType.MINIMUM; public boolean hasKeyType() { return ((bitField0_ & 0x00000010) == 0x00000010); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType getKeyType() { + public org.apache.hbase.protobuf.generated.CellProtos.CellType getKeyType() { return keyType_; } - public Builder setKeyType(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType value) { + public Builder setKeyType(org.apache.hbase.protobuf.generated.CellProtos.CellType value) { if (value == null) { throw new NullPointerException(); } @@ -8018,7 +7937,7 @@ public final class HBaseProtos { } public Builder clearKeyType() { bitField0_ = (bitField0_ & ~0x00000010); - keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM; + keyType_ = org.apache.hbase.protobuf.generated.CellProtos.CellType.MINIMUM; onChanged(); return this; } @@ -11740,58 +11659,55 @@ public final class HBaseProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\013hbase.proto\"\225\001\n\013TableSchema\022\014\n\004name\030\001 " + - "\001(\014\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair" + - "\022+\n\016columnFamilies\030\003 \003(\0132\023.ColumnFamilyS" + - "chema\022&\n\rconfiguration\030\004 \003(\0132\017.NameStrin" + - "gPair\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 \002(" + - "\014\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair\022&" + - "\n\rconfiguration\030\003 \003(\0132\017.NameStringPair\"s" + - "\n\nRegionInfo\022\020\n\010regionId\030\001 \002(\004\022\021\n\ttableN" + - "ame\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022\016\n\006endKey\030\004 " + - "\001(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\"\225\001\n\017", - "RegionSpecifier\0222\n\004type\030\001 \002(\0162$.RegionSp" + - "ecifier.RegionSpecifierType\022\r\n\005value\030\002 \002" + - "(\014\"?\n\023RegionSpecifierType\022\017\n\013REGION_NAME" + - "\020\001\022\027\n\023ENCODED_REGION_NAME\020\002\"\260\003\n\nRegionLo" + - "ad\022)\n\017regionSpecifier\030\001 \002(\0132\020.RegionSpec" + - "ifier\022\016\n\006stores\030\002 \001(\r\022\022\n\nstorefiles\030\003 \001(" + - "\r\022\037\n\027storeUncompressedSizeMB\030\004 \001(\r\022\027\n\017st" + - "orefileSizeMB\030\005 \001(\r\022\026\n\016memstoreSizeMB\030\006 " + - "\001(\r\022\034\n\024storefileIndexSizeMB\030\007 \001(\r\022\031\n\021rea" + - "dRequestsCount\030\010 \001(\004\022\032\n\022writeRequestsCou", - "nt\030\t \001(\004\022\032\n\022totalCompactingKVs\030\n \001(\004\022\033\n\023" + - "currentCompactedKVs\030\013 \001(\004\022\027\n\017rootIndexSi" + - "zeKB\030\014 \001(\r\022\036\n\026totalStaticIndexSizeKB\030\r \001" + - "(\r\022\036\n\026totalStaticBloomSizeKB\030\016 \001(\r\022\032\n\022co" + - "mpleteSequenceId\030\017 \001(\004\"\372\001\n\nServerLoad\022\030\n" + - "\020numberOfRequests\030\001 \001(\r\022\035\n\025totalNumberOf" + - "Requests\030\002 \001(\r\022\022\n\nusedHeapMB\030\003 \001(\r\022\021\n\tma" + - "xHeapMB\030\004 \001(\r\022 \n\013regionLoads\030\005 \003(\0132\013.Reg" + - "ionLoad\022\"\n\014coprocessors\030\006 \003(\0132\014.Coproces" + - "sor\022\027\n\017reportStartTime\030\007 \001(\004\022\025\n\rreportEn", - "dTime\030\010 \001(\004\022\026\n\016infoServerPort\030\t \001(\r\"%\n\tT" + - "imeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"0\n\006Fi" + - "lter\022\014\n\004name\030\001 \002(\t\022\030\n\020serializedFilter\030\002" + - " \001(\014\"w\n\010KeyValue\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030" + - "\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\021\n\ttimestamp\030\004 " + - "\001(\004\022\031\n\007keyType\030\005 \001(\0162\010.KeyType\022\r\n\005value\030" + - "\006 \001(\014\"?\n\nServerName\022\020\n\010hostName\030\001 \002(\t\022\014\n" + - "\004port\030\002 \001(\r\022\021\n\tstartCode\030\003 \001(\004\"\033\n\013Coproc" + - "essor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n" + - "\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesP", - "air\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016Byte" + - "sBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(" + - "\014\",\n\rNameInt64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005valu" + - "e\030\002 \001(\003\"\n\n\010EmptyMsg\"\032\n\007LongMsg\022\017\n\007longMs" + - "g\030\001 \002(\003*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS" + - "_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n" + - "\020GREATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_O" + - "P\020\006*_\n\007KeyType\022\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006" + - "DELETE\020\010\022\021\n\rDELETE_COLUMN\020\014\022\021\n\rDELETE_FA" + - "MILY\020\016\022\014\n\007MAXIMUM\020\377\001B>\n*org.apache.hadoo", - "p.hbase.protobuf.generatedB\013HBaseProtosH" + - "\001\240\001\001" + "\n\013hbase.proto\032\nCell.proto\"\225\001\n\013TableSchem" + + "a\022\014\n\004name\030\001 \001(\014\022#\n\nattributes\030\002 \003(\0132\017.By" + + "tesBytesPair\022+\n\016columnFamilies\030\003 \003(\0132\023.C" + + "olumnFamilySchema\022&\n\rconfiguration\030\004 \003(\013" + + "2\017.NameStringPair\"o\n\022ColumnFamilySchema\022" + + "\014\n\004name\030\001 \002(\014\022#\n\nattributes\030\002 \003(\0132\017.Byte" + + "sBytesPair\022&\n\rconfiguration\030\003 \003(\0132\017.Name" + + "StringPair\"s\n\nRegionInfo\022\020\n\010regionId\030\001 \002" + + "(\004\022\021\n\ttableName\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022" + + "\016\n\006endKey\030\004 \001(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005spli", + "t\030\006 \001(\010\"\225\001\n\017RegionSpecifier\0222\n\004type\030\001 \002(" + + "\0162$.RegionSpecifier.RegionSpecifierType\022" + + "\r\n\005value\030\002 \002(\014\"?\n\023RegionSpecifierType\022\017\n" + + "\013REGION_NAME\020\001\022\027\n\023ENCODED_REGION_NAME\020\002\"" + + "\260\003\n\nRegionLoad\022)\n\017regionSpecifier\030\001 \002(\0132" + + "\020.RegionSpecifier\022\016\n\006stores\030\002 \001(\r\022\022\n\nsto" + + "refiles\030\003 \001(\r\022\037\n\027storeUncompressedSizeMB" + + "\030\004 \001(\r\022\027\n\017storefileSizeMB\030\005 \001(\r\022\026\n\016memst" + + "oreSizeMB\030\006 \001(\r\022\034\n\024storefileIndexSizeMB\030" + + "\007 \001(\r\022\031\n\021readRequestsCount\030\010 \001(\004\022\032\n\022writ", + "eRequestsCount\030\t \001(\004\022\032\n\022totalCompactingK" + + "Vs\030\n \001(\004\022\033\n\023currentCompactedKVs\030\013 \001(\004\022\027\n" + + "\017rootIndexSizeKB\030\014 \001(\r\022\036\n\026totalStaticInd" + + "exSizeKB\030\r \001(\r\022\036\n\026totalStaticBloomSizeKB" + + "\030\016 \001(\r\022\032\n\022completeSequenceId\030\017 \001(\004\"\372\001\n\nS" + + "erverLoad\022\030\n\020numberOfRequests\030\001 \001(\r\022\035\n\025t" + + "otalNumberOfRequests\030\002 \001(\r\022\022\n\nusedHeapMB" + + "\030\003 \001(\r\022\021\n\tmaxHeapMB\030\004 \001(\r\022 \n\013regionLoads" + + "\030\005 \003(\0132\013.RegionLoad\022\"\n\014coprocessors\030\006 \003(" + + "\0132\014.Coprocessor\022\027\n\017reportStartTime\030\007 \001(\004", + "\022\025\n\rreportEndTime\030\010 \001(\004\022\026\n\016infoServerPor" + + "t\030\t \001(\r\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to" + + "\030\002 \001(\004\"0\n\006Filter\022\014\n\004name\030\001 \002(\t\022\030\n\020serial" + + "izedFilter\030\002 \001(\014\"x\n\010KeyValue\022\013\n\003row\030\001 \002(" + + "\014\022\016\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\021\n\t" + + "timestamp\030\004 \001(\004\022\032\n\007keyType\030\005 \001(\0162\t.CellT" + + "ype\022\r\n\005value\030\006 \001(\014\"?\n\nServerName\022\020\n\010host" + + "Name\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\021\n\tstartCode\030\003 " + + "\001(\004\"\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016Name" + + "StringPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\"", + ",\n\rNameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030" + + "\002 \001(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016" + + "\n\006second\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030" + + "\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\n\n\010EmptyMsg\"\032\n\007Long" + + "Msg\022\017\n\007longMsg\030\001 \002(\003*r\n\013CompareType\022\010\n\004L" + + "ESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tN" + + "OT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GREA" + + "TER\020\005\022\t\n\005NO_OP\020\006B>\n*org.apache.hadoop.hb" + + "ase.protobuf.generatedB\013HBaseProtosH\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -11940,6 +11856,7 @@ public final class HBaseProtos { com.google.protobuf.Descriptors.FileDescriptor .internalBuildGeneratedFileFrom(descriptorData, new com.google.protobuf.Descriptors.FileDescriptor[] { + org.apache.hbase.protobuf.generated.CellProtos.getDescriptor(), }, assigner); } diff --git a/hbase-protocol/src/main/protobuf/Aggregate.proto b/hbase-protocol/src/main/protobuf/Aggregate.proto index 5707e14..ca59f1f 100644 --- a/hbase-protocol/src/main/protobuf/Aggregate.proto +++ b/hbase-protocol/src/main/protobuf/Aggregate.proto @@ -59,4 +59,4 @@ service AggregateService { rpc getAvg (AggregateArgument) returns (AggregateResponse); rpc getStd (AggregateArgument) returns (AggregateResponse); rpc getMedian (AggregateArgument) returns (AggregateResponse); -} \ No newline at end of file +} diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto index d725b38..f52b516 100644 --- a/hbase-protocol/src/main/protobuf/Client.proto +++ b/hbase-protocol/src/main/protobuf/Client.proto @@ -18,6 +18,8 @@ // This file contains protocol buffers that are used for Client service. +import "Cell.proto"; + option java_package = "org.apache.hadoop.hbase.protobuf.generated"; option java_outer_classname = "ClientProtos"; option java_generic_services = true; @@ -51,7 +53,7 @@ message Get { } message Result { - repeated KeyValue keyValue = 1; + repeated Cell keyValue = 1; } /** diff --git a/hbase-protocol/src/main/protobuf/README.txt b/hbase-protocol/src/main/protobuf/README.txt index f979619..3575819 100644 --- a/hbase-protocol/src/main/protobuf/README.txt +++ b/hbase-protocol/src/main/protobuf/README.txt @@ -20,7 +20,7 @@ should be able to just copy and paste the below into a terminal and hit return fi for PROTO_FILE in $UNIX_PROTO_DIR/*.proto do - protoc -I$PROTO_DIR --java_out=$JAVA_DIR $PROTO_FILE + protoc -I$PROTO_DIR -I"../hbase-common/src/main/protobuf" --java_out=$JAVA_DIR $PROTO_FILE done After you've done the above, check it in and then check it in (or post a patch diff --git a/hbase-protocol/src/main/protobuf/hbase.proto b/hbase-protocol/src/main/protobuf/hbase.proto index aece7b0..2d4f410 100644 --- a/hbase-protocol/src/main/protobuf/hbase.proto +++ b/hbase-protocol/src/main/protobuf/hbase.proto @@ -17,6 +17,7 @@ */ // This file contains protocol buffers that are shared throughout HBase +import "Cell.proto"; option java_package = "org.apache.hadoop.hbase.protobuf.generated"; option java_outer_classname = "HBaseProtos"; @@ -201,21 +202,6 @@ enum CompareType { } /** - * The type of the key in a KeyValue. - */ -enum KeyType { - MINIMUM = 0; - PUT = 4; - - DELETE = 8; - DELETE_COLUMN = 12; - DELETE_FAMILY = 14; - - // MAXIMUM is used when searching; you look from maximum on down. - MAXIMUM = 255; -} - -/** * Protocol buffer version of KeyValue. * It doesn't have those transient parameters */ @@ -224,7 +210,7 @@ message KeyValue { required bytes family = 2; required bytes qualifier = 3; optional uint64 timestamp = 4; - optional KeyType keyType = 5; + optional CellType keyType = 5; optional bytes value = 6; } diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index 53c643b..90bf291 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -57,8 +57,8 @@ - org.apache.maven.plugins @@ -69,8 +69,8 @@ org/apache/hadoop/hbase/mapreduce/Driver - org/apache/jute/** @@ -207,7 +207,7 @@ - org.eclipse.m2e @@ -532,7 +532,7 @@ - @@ -550,8 +550,8 @@ - hadoop-1.0 diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Result.java index 9e15bbb..aab9074 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Result.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Result.java @@ -34,6 +34,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.SplitKeyValue; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hbase.Cell; /** * Single row result of a {@link Get} or {@link Scan} query.

    @@ -105,7 +106,8 @@ public class Result { * are already sorted * @param kvs List of KeyValues */ - public Result(List kvs) { + public Result(List kvs) { + // TODO: Here we presume the passed in Cells are KVs. One day this won't always be so. this(kvs.toArray(new KeyValue[kvs.size()])); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java index 925801a..590baf5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java @@ -16,14 +16,10 @@ */ package org.apache.hadoop.hbase.io.encoding; -import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; -import java.io.FilterOutputStream; import java.io.IOException; -import java.lang.reflect.Field; import java.nio.ByteBuffer; -import java.util.Arrays; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.KeyValue; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java index 4223dbe..c6e5adc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java @@ -17,13 +17,13 @@ */ package org.apache.hadoop.hbase.mapreduce; +import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.Serialization; import org.apache.hadoop.io.serializer.Serializer; @@ -45,43 +45,41 @@ public class KeyValueSerialization implements Serialization { } public static class KeyValueDeserializer implements Deserializer { - private InputStream is; + private DataInputStream dis; @Override public void close() throws IOException { - this.is.close(); + this.dis.close(); } @Override public KeyValue deserialize(KeyValue ignore) throws IOException { // I can't overwrite the passed in KV, not from a proto kv, not just yet. TODO - HBaseProtos.KeyValue proto = - HBaseProtos.KeyValue.parseDelimitedFrom(this.is); - return ProtobufUtil.toKeyValue(proto); + return KeyValue.create(this.dis); } @Override public void open(InputStream is) throws IOException { - this.is = is; + this.dis = new DataInputStream(is); } } public static class KeyValueSerializer implements Serializer { - private OutputStream os; + private DataOutputStream dos; @Override public void close() throws IOException { - this.os.close(); + this.dos.close(); } @Override public void open(OutputStream os) throws IOException { - this.os = os; + this.dos = new DataOutputStream(os); } @Override public void serialize(KeyValue kv) throws IOException { - ProtobufUtil.toKeyValue(kv).writeDelimitedTo(this.os); + KeyValue.write(kv, this.dos); } } } \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 08f8ba6..0c77ca4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -122,6 +122,8 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.token.Token; import org.apache.hbase.Cell; +import org.apache.hbase.CellTool; +import org.apache.hbase.protobuf.generated.CellProtos; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ListMultimap; @@ -831,7 +833,7 @@ public final class ProtobufUtil { Cell [] cells = result.raw(); if (cells != null) { for (Cell c : cells) { - builder.addKeyValue(toKeyValue(c)); + builder.addKeyValue(toCell(c)); } } return builder.build(); @@ -844,12 +846,12 @@ public final class ProtobufUtil { * @return the converted client Result */ public static Result toResult(final ClientProtos.Result proto) { - List values = proto.getKeyValueList(); - List keyValues = new ArrayList(values.size()); - for (HBaseProtos.KeyValue kv: values) { - keyValues.add(toKeyValue(kv)); + List values = proto.getKeyValueList(); + List cells = new ArrayList(values.size()); + for (CellProtos.Cell c: values) { + cells.add(toCell(c)); } - return new Result(keyValues); + return new Result(cells); } /** @@ -1792,33 +1794,31 @@ public final class ProtobufUtil { throw new IOException(se); } - public static HBaseProtos.KeyValue toKeyValue(final Cell kv) { + public static CellProtos.Cell toCell(final Cell kv) { // Doing this is going to kill us if we do it for all data passed. // St.Ack 20121205 - // TODO: Do a Cell version - HBaseProtos.KeyValue.Builder kvbuilder = HBaseProtos.KeyValue.newBuilder(); + CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder(); kvbuilder.setRow(ByteString.copyFrom(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength())); kvbuilder.setFamily(ByteString.copyFrom(kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength())); kvbuilder.setQualifier(ByteString.copyFrom(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength())); - kvbuilder.setKeyType(HBaseProtos.KeyType.valueOf(kv.getTypeByte())); + kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte())); kvbuilder.setTimestamp(kv.getTimestamp()); kvbuilder.setValue(ByteString.copyFrom(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); return kvbuilder.build(); } - public static KeyValue toKeyValue(final HBaseProtos.KeyValue kv) { + public static Cell toCell(final CellProtos.Cell cell) { // Doing this is going to kill us if we do it for all data passed. // St.Ack 20121205 - // TODO: Do a Cell version - return new KeyValue(kv.getRow().toByteArray(), - kv.getFamily().toByteArray(), - kv.getQualifier().toByteArray(), - kv.getTimestamp(), - KeyValue.Type.codeToType((byte)kv.getKeyType().getNumber()), - kv.getValue().toByteArray()); + return CellTool.createCell(cell.getRow().toByteArray(), + cell.getFamily().toByteArray(), + cell.getQualifier().toByteArray(), + cell.getTimestamp(), + (byte)cell.getCellType().getNumber(), + cell.getValue().toByteArray()); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java index db1f7aa..515516e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.cell.CellComparator; +import org.apache.hbase.CellComparator; import org.junit.Test; import org.junit.experimental.categories.Category; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 49a72c4..f28c24f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -90,7 +90,7 @@ import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.PairOfSameType; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hbase.cell.CellComparator; +import org.apache.hbase.CellComparator; import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java index 23d71d3..9eb531f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.cell.CellComparator; +import org.apache.hbase.CellComparator; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith;