| , 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:
- *
- * - In a DataBlockScanner, the AFTER_LAST position indicates to the parent StoreFileScanner that
- * it should load the next block.
- * - In a StoreFileScanner, the AFTER_LAST position indicates that the file has been exhausted.
- * - In a RegionScanner, the AFTER_LAST position indicates that the scanner should move to the
- * next region.
- *
- */
-@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..d3f5bff
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hbase/codec/CodecException.java
@@ -0,0 +1,22 @@
+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/codec/OldSchoolKeyValueDecoder.java b/hbase-common/src/main/java/org/apache/hbase/codec/OldSchoolKeyValueDecoder.java
new file mode 100644
index 0000000..2cc190f
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hbase/codec/OldSchoolKeyValueDecoder.java
@@ -0,0 +1,49 @@
+package org.apache.hbase.codec;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hbase.Cell;
+import org.apache.hbase.CellScanner;
+
+/**
+ * Depends on {@link InputStream#available()}
+ * @see OldSchoolKeyValueEncoder
+ */
+public class OldSchoolKeyValueDecoder implements CellScanner {
+ private final DataInputStream in;
+ private boolean hasNext = true;
+ private Cell current = null;
+
+ public OldSchoolKeyValueDecoder(final InputStream in) {
+ this.in = new DataInputStream(in);
+ }
+
+ @Override
+ public boolean next() {
+ if (!this.hasNext) return !this.hasNext;
+ try {
+ if (this.in.available() <= 0) {
+ this.hasNext = false;
+ return this.hasNext;
+ }
+ int length = this.in.readInt();
+ this.current = KeyValue.create(length, this.in);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ return this.hasNext;
+ }
+
+ @Override
+ public Cell get() {
+ return this.current;
+ }
+
+ @Override
+ public Cell getDeepCopy() {
+ return get();
+ }
+}
\ No newline at end of file
diff --git a/hbase-common/src/main/java/org/apache/hbase/codec/OldSchoolKeyValueEncoder.java b/hbase-common/src/main/java/org/apache/hbase/codec/OldSchoolKeyValueEncoder.java
new file mode 100644
index 0000000..f79ef86
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hbase/codec/OldSchoolKeyValueEncoder.java
@@ -0,0 +1,63 @@
+package org.apache.hbase.codec;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hbase.Cell;
+import org.apache.hbase.io.CellOutputStream;
+
+/**
+ * Encodes by casting Cell to KeyValue and writing out the backing array with a length prefix.
+ * This is how KVs were serialized in Puts, Deletes and Results pre-0.96. Its what would
+ * happen if you called the Writable#write KeyValue implementation. This encoder will fail
+ * if the passed Cell is not an old-school pre-0.96 KeyValue. Does not copy bytes writing.
+ * It just writes them direct to the passed stream.
+ *
+ * If you wrote two KeyValues to this encoder, it would look like this in the stream:
+ *
+ * length-of-KeyValue1 // A java int with the length of KeyValue1 backing array
+ * KeyValue1 backing array filled with a KeyValue serialized in its particular format
+ * length-of-KeyValue2
+ * KeyValue2 backing array
+ *
+ * @see OldSchoolKeyValueDecoder
+ */
+public class OldSchoolKeyValueEncoder implements CellOutputStream {
+ // Need to be able to write java types such as int and long so need DataOutput.
+ // Want to stream too so DataOutputStream.
+ private final DataOutputStream out;
+ private boolean flushed = false;
+
+ public OldSchoolKeyValueEncoder(final OutputStream out) {
+ this.out = new DataOutputStream(out);
+ }
+
+ @Override
+ public void write(Cell cell) throws IOException {
+ if (this.flushed) throw new CodecException("Flushed, done");
+ // This is crass and will not work when KV changes
+ try {
+ KeyValue.write((KeyValue)cell, this.out);
+ } catch (IOException e) {
+ throw new CodecException(e);
+ }
+ }
+
+ /**
+ * Calling flush 'finishes' this encoder. Subsequent calls
+ * to {@link #write(Cell)} will throw exception.
+ */
+ @Override
+ public void flush() throws IOException {
+ if (this.flushed) return;
+ this.flushed = true;
+ // Write out an int whose value is zero as end of stream.
+ try {
+ this.out.flush();
+ } catch (IOException e) {
+ throw new CodecException(e);
+ }
+ }
+}
\ 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-common/src/test/java/org/apache/hbase/codec/TestOldSchoolKeyValueCodec.java b/hbase-common/src/test/java/org/apache/hbase/codec/TestOldSchoolKeyValueCodec.java
new file mode 100644
index 0000000..d8265b2
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hbase/codec/TestOldSchoolKeyValueCodec.java
@@ -0,0 +1,95 @@
+package org.apache.hbase.codec;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.codec.OldSchoolKeyValueDecoder;
+import org.apache.hbase.codec.OldSchoolKeyValueEncoder;
+import org.junit.Test;
+
+import com.google.common.io.CountingInputStream;
+import com.google.common.io.CountingOutputStream;
+
+public class TestOldSchoolKeyValueCodec {
+
+ @Test
+ public void testEmptyWorks() throws IOException {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ CountingOutputStream cos = new CountingOutputStream(baos);
+ DataOutputStream dos = new DataOutputStream(cos);
+ OldSchoolKeyValueEncoder encoder = new OldSchoolKeyValueEncoder(dos);
+ encoder.flush();
+ dos.close();
+ long offset = cos.getCount();
+ assertEquals(0, offset);
+ CountingInputStream cis = new CountingInputStream(new ByteArrayInputStream(baos.toByteArray()));
+ DataInputStream dis = new DataInputStream(cis);
+ OldSchoolKeyValueDecoder decoder = new OldSchoolKeyValueDecoder(dis);
+ assertFalse(decoder.next());
+ dis.close();
+ assertEquals(0, cis.getCount());
+ }
+
+ @Test
+ public void testOne() throws IOException {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ CountingOutputStream cos = new CountingOutputStream(baos);
+ DataOutputStream dos = new DataOutputStream(cos);
+ OldSchoolKeyValueEncoder encoder = new OldSchoolKeyValueEncoder(dos);
+ final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"), Bytes.toBytes("v"));
+ final long length = kv.getLength() + Bytes.SIZEOF_INT;
+ encoder.write(kv);
+ encoder.flush();
+ dos.close();
+ long offset = cos.getCount();
+ assertEquals(length, offset);
+ CountingInputStream cis = new CountingInputStream(new ByteArrayInputStream(baos.toByteArray()));
+ DataInputStream dis = new DataInputStream(cis);
+ OldSchoolKeyValueDecoder decoder = new OldSchoolKeyValueDecoder(dis);
+ assertTrue(decoder.next()); // First read should pull in the KV
+ assertFalse(decoder.next()); // Second read should trip over the end-of-stream marker and return false
+ dis.close();
+ assertEquals(length, cis.getCount());
+ }
+
+ @Test
+ public void testThree() throws IOException {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ CountingOutputStream cos = new CountingOutputStream(baos);
+ DataOutputStream dos = new DataOutputStream(cos);
+ OldSchoolKeyValueEncoder encoder = new OldSchoolKeyValueEncoder(dos);
+ final KeyValue kv1 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("1"), Bytes.toBytes("1"));
+ final KeyValue kv2 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("2"), Bytes.toBytes("2"));
+ final KeyValue kv3 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("3"), Bytes.toBytes("3"));
+ final long length = kv1.getLength() + Bytes.SIZEOF_INT;
+ encoder.write(kv1);
+ encoder.write(kv2);
+ encoder.write(kv3);
+ encoder.flush();
+ dos.close();
+ long offset = cos.getCount();
+ assertEquals(length * 3, offset);
+ CountingInputStream cis = new CountingInputStream(new ByteArrayInputStream(baos.toByteArray()));
+ DataInputStream dis = new DataInputStream(cis);
+ OldSchoolKeyValueDecoder decoder = new OldSchoolKeyValueDecoder(dis);
+ assertTrue(decoder.next());
+ KeyValue kv = (KeyValue)decoder.get();
+ assertTrue(kv1.equals(kv));
+ assertTrue(decoder.next());
+ kv = (KeyValue)decoder.get();
+ assertTrue(kv2.equals(kv));
+ assertTrue(decoder.next());
+ kv = (KeyValue)decoder.get();
+ assertTrue(kv3.equals(kv));
+ assertFalse(decoder.next());
+ dis.close();
+ assertEquals((length * 3), cis.getCount());
+ }
+}
\ 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..652b2a1 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.hadoop.hbase.protobuf.generated.HBaseProtos.Cell getKeyValue(int index);
int getKeyValueCount();
- java.util.List extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder>
+ java.util.List extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder>
getKeyValueOrBuilderList();
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder getKeyValueOrBuilder(
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.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 extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder>
+ public java.util.List extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder>
getKeyValueOrBuilderList() {
return keyValue_;
}
public int getKeyValueCount() {
return keyValue_.size();
}
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue getKeyValue(int index) {
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell getKeyValue(int index) {
return keyValue_.get(index);
}
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder getKeyValueOrBuilder(
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.Cell, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.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 extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue> values) {
+ java.lang.Iterable extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell> 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.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder getKeyValueBuilder(
int index) {
return getKeyValueFieldBuilder().getBuilder(index);
}
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder getKeyValueOrBuilder(
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder getKeyValueOrBuilder(
int index) {
if (keyValueBuilder_ == null) {
return keyValue_.get(index); } else {
return keyValueBuilder_.getMessageOrBuilder(index);
}
}
- public java.util.List extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder>
+ public java.util.List extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder>
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.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder addKeyValueBuilder() {
return getKeyValueFieldBuilder().addBuilder(
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.getDefaultInstance());
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.getDefaultInstance());
}
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder addKeyValueBuilder(
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder addKeyValueBuilder(
int index) {
return getKeyValueFieldBuilder().addBuilder(
- index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.getDefaultInstance());
+ index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.Cell, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.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.hadoop.hbase.protobuf.generated.HBaseProtos.Cell, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.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()) {
@@ -21068,83 +20996,83 @@ public final class ClientProtos {
"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\006Result\022\027\n\010keyValue\030\001 \003(\0132\005.Cell\"r\n\nGe" +
+ "tRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifi",
+ "er\022\021\n\003get\030\002 \002(\0132\004.Get\022\030\n\020closestRowBefor" +
+ "e\030\003 \001(\010\022\025\n\rexistenceOnly\030\004 \001(\010\"w\n\017MultiG" +
+ "etRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecif" +
+ "ier\022\021\n\003get\030\002 \003(\0132\004.Get\022\030\n\020closestRowBefo" +
+ "re\030\003 \001(\010\022\025\n\rexistenceOnly\030\004 \001(\010\"6\n\013GetRe" +
+ "sponse\022\027\n\006result\030\001 \001(\0132\007.Result\022\016\n\006exist" +
+ "s\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\tConditio" +
+ "n\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002 \002(\014\022\021\n\tqualif" +
+ "ier\030\003 \002(\014\022!\n\013compareType\030\004 \002(\0162\014.Compare",
+ "Type\022\037\n\ncomparator\030\005 \002(\0132\013.Comparator\"\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\006family" +
+ "\030\001 \002(\014\022:\n\016qualifierValue\030\002 \003(\0132\".Mutate." +
+ "ColumnValue.QualifierValue\032m\n\016QualifierV" +
+ "alue\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\006APPEN" +
+ "D\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_FA" +
+ "MILY\020\002\"i\n\rMutateRequest\022 \n\006region\030\001 \002(\0132" +
+ "\020.RegionSpecifier\022\027\n\006mutate\030\002 \002(\0132\007.Muta" +
+ "te\022\035\n\tcondition\030\003 \001(\0132\n.Condition\"<\n\016Mut" +
+ "ateResponse\022\027\n\006result\030\001 \001(\0132\007.Result\022\021\n\t" +
+ "processed\030\002 \001(\010\"\307\002\n\004Scan\022\027\n\006column\030\001 \003(\013",
+ "2\007.Column\022!\n\tattribute\030\002 \003(\0132\016.NameBytes" +
+ "Pair\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\ttimeRange\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\tbatchSize\030" +
+ "\t \001(\r\022\025\n\rmaxResultSize\030\n \001(\004\022\022\n\nstoreLim" +
+ "it\030\013 \001(\r\022\023\n\013storeOffset\030\014 \001(\r\022\"\n\032loadCol" +
+ "umnFamiliesOnDemand\030\r \001(\010\"\230\001\n\013ScanReques" +
+ "t\022 \n\006region\030\001 \001(\0132\020.RegionSpecifier\022\023\n\004s" +
+ "can\030\002 \001(\0132\005.Scan\022\021\n\tscannerId\030\003 \001(\004\022\024\n\014n",
+ "umberOfRows\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\006" +
+ "result\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\017re" +
+ "sultSizeBytes\030\005 \001(\004\"\260\001\n\024BulkLoadHFileReq" +
+ "uest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\0224" +
+ "\n\nfamilyPath\030\002 \003(\0132 .BulkLoadHFileReques" +
+ "t.FamilyPath\022\024\n\014assignSeqNum\030\003 \001(\010\032*\n\nFa" +
+ "milyPath\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\031CoprocessorServiceRe" +
+ "quest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022" +
+ "%\n\004call\030\002 \002(\0132\027.CoprocessorServiceCall\"]" +
+ "\n\032CoprocessorServiceResponse\022 \n\006region\030\001" +
+ " \002(\0132\020.RegionSpecifier\022\035\n\005value\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\014Acti" +
+ "onResult\022\026\n\005value\030\001 \001(\0132\007.Result\022!\n\texce",
+ "ption\030\002 \001(\0132\016.NameBytesPair\"^\n\014MultiRequ" +
+ "est\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\034\n" +
+ "\006action\030\002 \003(\0132\014.MultiAction\022\016\n\006atomic\030\003 " +
+ "\001(\010\".\n\rMultiResponse\022\035\n\006result\030\001 \003(\0132\r.A" +
+ "ctionResult2\342\002\n\rClientService\022 \n\003get\022\013.G" +
+ "etRequest\032\014.GetResponse\022/\n\010multiGet\022\020.Mu" +
+ "ltiGetRequest\032\021.MultiGetResponse\022)\n\006muta" +
+ "te\022\016.MutateRequest\032\017.MutateResponse\022#\n\004s" +
+ "can\022\014.ScanRequest\032\r.ScanResponse\022>\n\rbulk" +
+ "LoadHFile\022\025.BulkLoadHFileRequest\032\026.BulkL",
+ "oadHFileResponse\022F\n\013execService\022\032.Coproc" +
+ "essorServiceRequest\032\033.CoprocessorService" +
+ "Response\022&\n\005multi\022\r.MultiRequest\032\016.Multi" +
+ "ResponseBB\n*org.apache.hadoop.hbase.prot" +
+ "obuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
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..fdf35ca 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,7 +92,7 @@ public final class HBaseProtos {
// @@protoc_insertion_point(enum_scope:CompareType)
}
- public enum KeyType
+ public enum CellType
implements com.google.protobuf.ProtocolMessageEnum {
MINIMUM(0, 0),
PUT(1, 4),
@@ -112,7 +112,7 @@ public final class HBaseProtos {
public final int getNumber() { return value; }
- public static KeyType valueOf(int value) {
+ public static CellType valueOf(int value) {
switch (value) {
case 0: return MINIMUM;
case 4: return PUT;
@@ -124,15 +124,15 @@ public final class HBaseProtos {
}
}
- public static com.google.protobuf.Internal.EnumLiteMap
+ public static com.google.protobuf.Internal.EnumLiteMap
internalGetValueMap() {
return internalValueMap;
}
- private static com.google.protobuf.Internal.EnumLiteMap
+ private static com.google.protobuf.Internal.EnumLiteMap
internalValueMap =
- new com.google.protobuf.Internal.EnumLiteMap() {
- public KeyType findValueByNumber(int number) {
- return KeyType.valueOf(number);
+ new com.google.protobuf.Internal.EnumLiteMap() {
+ public CellType findValueByNumber(int number) {
+ return CellType.valueOf(number);
}
};
@@ -149,11 +149,11 @@ public final class HBaseProtos {
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor().getEnumTypes().get(1);
}
- private static final KeyType[] VALUES = {
+ private static final CellType[] VALUES = {
MINIMUM, PUT, DELETE, DELETE_COLUMN, DELETE_FAMILY, MAXIMUM,
};
- public static KeyType valueOf(
+ public static CellType valueOf(
com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
if (desc.getType() != getDescriptor()) {
throw new java.lang.IllegalArgumentException(
@@ -165,12 +165,12 @@ public final class HBaseProtos {
private final int index;
private final int value;
- private KeyType(int index, int value) {
+ private CellType(int index, int value) {
this.index = index;
this.value = value;
}
- // @@protoc_insertion_point(enum_scope:KeyType)
+ // @@protoc_insertion_point(enum_scope:CellType)
}
public interface TableSchemaOrBuilder
@@ -7312,18 +7312,18 @@ public final class HBaseProtos {
// @@protoc_insertion_point(class_scope:Filter)
}
- public interface KeyValueOrBuilder
+ public interface CellOrBuilder
extends com.google.protobuf.MessageOrBuilder {
- // required bytes row = 1;
+ // optional bytes row = 1;
boolean hasRow();
com.google.protobuf.ByteString getRow();
- // required bytes family = 2;
+ // optional bytes family = 2;
boolean hasFamily();
com.google.protobuf.ByteString getFamily();
- // required bytes qualifier = 3;
+ // optional bytes qualifier = 3;
boolean hasQualifier();
com.google.protobuf.ByteString getQualifier();
@@ -7331,44 +7331,44 @@ public final class HBaseProtos {
boolean hasTimestamp();
long getTimestamp();
- // optional .KeyType keyType = 5;
- boolean hasKeyType();
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType getKeyType();
+ // optional .CellType cellType = 5;
+ boolean hasCellType();
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType getCellType();
// optional bytes value = 6;
boolean hasValue();
com.google.protobuf.ByteString getValue();
}
- public static final class KeyValue extends
+ public static final class Cell extends
com.google.protobuf.GeneratedMessage
- implements KeyValueOrBuilder {
- // Use KeyValue.newBuilder() to construct.
- private KeyValue(Builder builder) {
+ implements CellOrBuilder {
+ // Use Cell.newBuilder() to construct.
+ private Cell(Builder builder) {
super(builder);
}
- private KeyValue(boolean noInit) {}
+ private Cell(boolean noInit) {}
- private static final KeyValue defaultInstance;
- public static KeyValue getDefaultInstance() {
+ private static final Cell defaultInstance;
+ public static Cell getDefaultInstance() {
return defaultInstance;
}
- public KeyValue getDefaultInstanceForType() {
+ public Cell getDefaultInstanceForType() {
return defaultInstance;
}
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_KeyValue_descriptor;
+ return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Cell_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_KeyValue_fieldAccessorTable;
+ return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Cell_fieldAccessorTable;
}
private int bitField0_;
- // required bytes row = 1;
+ // optional bytes row = 1;
public static final int ROW_FIELD_NUMBER = 1;
private com.google.protobuf.ByteString row_;
public boolean hasRow() {
@@ -7378,7 +7378,7 @@ public final class HBaseProtos {
return row_;
}
- // required bytes family = 2;
+ // optional bytes family = 2;
public static final int FAMILY_FIELD_NUMBER = 2;
private com.google.protobuf.ByteString family_;
public boolean hasFamily() {
@@ -7388,7 +7388,7 @@ public final class HBaseProtos {
return family_;
}
- // required bytes qualifier = 3;
+ // optional bytes qualifier = 3;
public static final int QUALIFIER_FIELD_NUMBER = 3;
private com.google.protobuf.ByteString qualifier_;
public boolean hasQualifier() {
@@ -7408,14 +7408,14 @@ public final class HBaseProtos {
return timestamp_;
}
- // optional .KeyType keyType = 5;
- public static final int KEYTYPE_FIELD_NUMBER = 5;
- private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType keyType_;
- public boolean hasKeyType() {
+ // optional .CellType cellType = 5;
+ public static final int CELLTYPE_FIELD_NUMBER = 5;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType cellType_;
+ public boolean hasCellType() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType getKeyType() {
- return keyType_;
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType getCellType() {
+ return cellType_;
}
// optional bytes value = 6;
@@ -7433,7 +7433,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;
+ cellType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM;
value_ = com.google.protobuf.ByteString.EMPTY;
}
private byte memoizedIsInitialized = -1;
@@ -7441,18 +7441,6 @@ public final class HBaseProtos {
byte isInitialized = memoizedIsInitialized;
if (isInitialized != -1) return isInitialized == 1;
- if (!hasRow()) {
- memoizedIsInitialized = 0;
- return false;
- }
- if (!hasFamily()) {
- memoizedIsInitialized = 0;
- return false;
- }
- if (!hasQualifier()) {
- memoizedIsInitialized = 0;
- return false;
- }
memoizedIsInitialized = 1;
return true;
}
@@ -7473,7 +7461,7 @@ public final class HBaseProtos {
output.writeUInt64(4, timestamp_);
}
if (((bitField0_ & 0x00000010) == 0x00000010)) {
- output.writeEnum(5, keyType_.getNumber());
+ output.writeEnum(5, cellType_.getNumber());
}
if (((bitField0_ & 0x00000020) == 0x00000020)) {
output.writeBytes(6, value_);
@@ -7505,7 +7493,7 @@ public final class HBaseProtos {
}
if (((bitField0_ & 0x00000010) == 0x00000010)) {
size += com.google.protobuf.CodedOutputStream
- .computeEnumSize(5, keyType_.getNumber());
+ .computeEnumSize(5, cellType_.getNumber());
}
if (((bitField0_ & 0x00000020) == 0x00000020)) {
size += com.google.protobuf.CodedOutputStream
@@ -7528,10 +7516,10 @@ public final class HBaseProtos {
if (obj == this) {
return true;
}
- if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue)) {
+ if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell)) {
return super.equals(obj);
}
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue) obj;
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell) obj;
boolean result = true;
result = result && (hasRow() == other.hasRow());
@@ -7554,10 +7542,10 @@ public final class HBaseProtos {
result = result && (getTimestamp()
== other.getTimestamp());
}
- result = result && (hasKeyType() == other.hasKeyType());
- if (hasKeyType()) {
+ result = result && (hasCellType() == other.hasCellType());
+ if (hasCellType()) {
result = result &&
- (getKeyType() == other.getKeyType());
+ (getCellType() == other.getCellType());
}
result = result && (hasValue() == other.hasValue());
if (hasValue()) {
@@ -7589,9 +7577,9 @@ public final class HBaseProtos {
hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getTimestamp());
}
- if (hasKeyType()) {
- hash = (37 * hash) + KEYTYPE_FIELD_NUMBER;
- hash = (53 * hash) + hashEnum(getKeyType());
+ if (hasCellType()) {
+ hash = (37 * hash) + CELLTYPE_FIELD_NUMBER;
+ hash = (53 * hash) + hashEnum(getCellType());
}
if (hasValue()) {
hash = (37 * hash) + VALUE_FIELD_NUMBER;
@@ -7601,41 +7589,41 @@ public final class HBaseProtos {
return hash;
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseFrom(
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseFrom(
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(
com.google.protobuf.ByteString data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed();
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseFrom(byte[] data)
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseFrom(
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(
byte[] data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed();
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseFrom(java.io.InputStream input)
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(java.io.InputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseFrom(
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseDelimitedFrom(java.io.InputStream input)
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException {
Builder builder = newBuilder();
if (builder.mergeDelimitedFrom(input)) {
@@ -7644,7 +7632,7 @@ public final class HBaseProtos {
return null;
}
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseDelimitedFrom(
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseDelimitedFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
@@ -7655,12 +7643,12 @@ public final class HBaseProtos {
return null;
}
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseFrom(
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(
com.google.protobuf.CodedInputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
- public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue parseFrom(
+ public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
@@ -7670,7 +7658,7 @@ public final class HBaseProtos {
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue prototype) {
+ public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
@@ -7683,18 +7671,18 @@ public final class HBaseProtos {
}
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder
- implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder {
+ implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_KeyValue_descriptor;
+ return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Cell_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_KeyValue_fieldAccessorTable;
+ return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Cell_fieldAccessorTable;
}
- // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.newBuilder()
+ // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.newBuilder()
private Builder() {
maybeForceBuilderInitialization();
}
@@ -7721,7 +7709,7 @@ public final class HBaseProtos {
bitField0_ = (bitField0_ & ~0x00000004);
timestamp_ = 0L;
bitField0_ = (bitField0_ & ~0x00000008);
- keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM;
+ cellType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM;
bitField0_ = (bitField0_ & ~0x00000010);
value_ = com.google.protobuf.ByteString.EMPTY;
bitField0_ = (bitField0_ & ~0x00000020);
@@ -7734,24 +7722,24 @@ public final class HBaseProtos {
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
- return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.getDescriptor();
+ return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.getDescriptor();
}
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue getDefaultInstanceForType() {
- return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.getDefaultInstance();
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.getDefaultInstance();
}
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue build() {
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue result = buildPartial();
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell build() {
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
}
return result;
}
- private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue buildParsed()
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue result = buildPartial();
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(
result).asInvalidProtocolBufferException();
@@ -7759,8 +7747,8 @@ public final class HBaseProtos {
return result;
}
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue buildPartial() {
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue(this);
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell buildPartial() {
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell(this);
int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -7782,7 +7770,7 @@ public final class HBaseProtos {
if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
to_bitField0_ |= 0x00000010;
}
- result.keyType_ = keyType_;
+ result.cellType_ = cellType_;
if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
to_bitField0_ |= 0x00000020;
}
@@ -7793,16 +7781,16 @@ public final class HBaseProtos {
}
public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue) {
- return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue)other);
+ if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell) {
+ return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell)other);
} else {
super.mergeFrom(other);
return this;
}
}
- public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue other) {
- if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.getDefaultInstance()) return this;
+ public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell other) {
+ if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.getDefaultInstance()) return this;
if (other.hasRow()) {
setRow(other.getRow());
}
@@ -7815,8 +7803,8 @@ public final class HBaseProtos {
if (other.hasTimestamp()) {
setTimestamp(other.getTimestamp());
}
- if (other.hasKeyType()) {
- setKeyType(other.getKeyType());
+ if (other.hasCellType()) {
+ setCellType(other.getCellType());
}
if (other.hasValue()) {
setValue(other.getValue());
@@ -7826,18 +7814,6 @@ public final class HBaseProtos {
}
public final boolean isInitialized() {
- if (!hasRow()) {
-
- return false;
- }
- if (!hasFamily()) {
-
- return false;
- }
- if (!hasQualifier()) {
-
- return false;
- }
return true;
}
@@ -7886,12 +7862,12 @@ 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.hadoop.hbase.protobuf.generated.HBaseProtos.CellType value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.valueOf(rawValue);
if (value == null) {
unknownFields.mergeVarintField(5, rawValue);
} else {
bitField0_ |= 0x00000010;
- keyType_ = value;
+ cellType_ = value;
}
break;
}
@@ -7906,7 +7882,7 @@ public final class HBaseProtos {
private int bitField0_;
- // required bytes row = 1;
+ // optional bytes row = 1;
private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY;
public boolean hasRow() {
return ((bitField0_ & 0x00000001) == 0x00000001);
@@ -7930,7 +7906,7 @@ public final class HBaseProtos {
return this;
}
- // required bytes family = 2;
+ // optional bytes family = 2;
private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
public boolean hasFamily() {
return ((bitField0_ & 0x00000002) == 0x00000002);
@@ -7954,7 +7930,7 @@ public final class HBaseProtos {
return this;
}
- // required bytes qualifier = 3;
+ // optional bytes qualifier = 3;
private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY;
public boolean hasQualifier() {
return ((bitField0_ & 0x00000004) == 0x00000004);
@@ -7999,26 +7975,26 @@ 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;
- public boolean hasKeyType() {
+ // optional .CellType cellType = 5;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType cellType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM;
+ public boolean hasCellType() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType getKeyType() {
- return keyType_;
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType getCellType() {
+ return cellType_;
}
- public Builder setKeyType(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType value) {
+ public Builder setCellType(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00000010;
- keyType_ = value;
+ cellType_ = value;
onChanged();
return this;
}
- public Builder clearKeyType() {
+ public Builder clearCellType() {
bitField0_ = (bitField0_ & ~0x00000010);
- keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM;
+ cellType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM;
onChanged();
return this;
}
@@ -8047,15 +8023,15 @@ public final class HBaseProtos {
return this;
}
- // @@protoc_insertion_point(builder_scope:KeyValue)
+ // @@protoc_insertion_point(builder_scope:Cell)
}
static {
- defaultInstance = new KeyValue(true);
+ defaultInstance = new Cell(true);
defaultInstance.initFields();
}
- // @@protoc_insertion_point(class_scope:KeyValue)
+ // @@protoc_insertion_point(class_scope:Cell)
}
public interface ServerNameOrBuilder
@@ -11687,10 +11663,10 @@ public final class HBaseProtos {
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_Filter_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
- internal_static_KeyValue_descriptor;
+ internal_static_Cell_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_KeyValue_fieldAccessorTable;
+ internal_static_Cell_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
internal_static_ServerName_descriptor;
private static
@@ -11773,25 +11749,25 @@ public final class HBaseProtos {
"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"
+ " \001(\014\"u\n\004Cell\022\013\n\003row\030\001 \001(\014\022\016\n\006family\030\002 \001(" +
+ "\014\022\021\n\tqualifier\030\003 \001(\014\022\021\n\ttimestamp\030\004 \001(\004\022" +
+ "\033\n\010cellType\030\005 \001(\0162\t.CellType\022\r\n\005value\030\006 " +
+ "\001(\014\"?\n\nServerName\022\020\n\010hostName\030\001 \002(\t\022\014\n\004p" +
+ "ort\030\002 \001(\r\022\021\n\tstartCode\030\003 \001(\004\"\033\n\013Coproces" +
+ "sor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004n" +
+ "ame\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPai",
+ "r\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesB" +
+ "ytesPair\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\007LongMsg\022\017\n\007longMsg\030" +
+ "\001 \002(\003*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_O" +
+ "R_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020G" +
+ "REATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020" +
+ "\006*`\n\010CellType\022\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006D" +
+ "ELETE\020\010\022\021\n\rDELETE_COLUMN\020\014\022\021\n\rDELETE_FAM" +
+ "ILY\020\016\022\014\n\007MAXIMUM\020\377\001B>\n*org.apache.hadoop",
+ ".hbase.protobuf.generatedB\013HBaseProtosH\001" +
+ "\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -11862,14 +11838,14 @@ public final class HBaseProtos {
new java.lang.String[] { "Name", "SerializedFilter", },
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.class,
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder.class);
- internal_static_KeyValue_descriptor =
+ internal_static_Cell_descriptor =
getDescriptor().getMessageTypes().get(8);
- internal_static_KeyValue_fieldAccessorTable = new
+ internal_static_Cell_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_KeyValue_descriptor,
- new java.lang.String[] { "Row", "Family", "Qualifier", "Timestamp", "KeyType", "Value", },
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.class,
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder.class);
+ internal_static_Cell_descriptor,
+ new java.lang.String[] { "Row", "Family", "Qualifier", "Timestamp", "CellType", "Value", },
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.class,
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder.class);
internal_static_ServerName_descriptor =
getDescriptor().getMessageTypes().get(9);
internal_static_ServerName_fieldAccessorTable = new
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..096a9e7 100644
--- a/hbase-protocol/src/main/protobuf/Client.proto
+++ b/hbase-protocol/src/main/protobuf/Client.proto
@@ -51,7 +51,7 @@ message Get {
}
message Result {
- repeated KeyValue keyValue = 1;
+ repeated Cell keyValue = 1;
}
/**
diff --git a/hbase-protocol/src/main/protobuf/hbase.proto b/hbase-protocol/src/main/protobuf/hbase.proto
index aece7b0..86e6f38 100644
--- a/hbase-protocol/src/main/protobuf/hbase.proto
+++ b/hbase-protocol/src/main/protobuf/hbase.proto
@@ -201,9 +201,9 @@ enum CompareType {
}
/**
- * The type of the key in a KeyValue.
+ * The type of the key in a Cell
*/
-enum KeyType {
+enum CellType {
MINIMUM = 0;
PUT = 4;
@@ -216,15 +216,14 @@ enum KeyType {
}
/**
- * Protocol buffer version of KeyValue.
- * It doesn't have those transient parameters
+ * Protocol buffer version of Cell.
*/
-message KeyValue {
- required bytes row = 1;
- required bytes family = 2;
- required bytes qualifier = 3;
+message Cell {
+ optional bytes row = 1;
+ optional bytes family = 2;
+ optional bytes qualifier = 3;
optional uint64 timestamp = 4;
- optional KeyType keyType = 5;
+ optional CellType cellType = 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 extends Cell> 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/protobuf/ProtobufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 08f8ba6..ae266fd 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,7 @@ 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 com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
@@ -831,7 +832,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 +845,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 (HBaseProtos.Cell c: values) {
+ cells.add(toCell(c));
}
- return new Result(keyValues);
+ return new Result(cells);
}
/**
@@ -1792,33 +1793,33 @@ public final class ProtobufUtil {
throw new IOException(se);
}
- public static HBaseProtos.KeyValue toKeyValue(final Cell kv) {
+ public static HBaseProtos.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();
+ HBaseProtos.Cell.Builder kvbuilder = HBaseProtos.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(HBaseProtos.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 HBaseProtos.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;
| | | |