diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java index d081f03..5f27aaa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java @@ -19,10 +19,10 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; import java.util.ArrayList; import java.util.Arrays; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java index 30238ef..0db0167 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java @@ -21,10 +21,10 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; import java.io.IOException; import java.util.ArrayList; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 2d3354c..bbf5e78 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -51,7 +52,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hbase.Cell; import java.io.Closeable; import java.io.IOException; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java index 4298026..6df3626 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java @@ -25,12 +25,12 @@ import java.util.Map; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; /** * Used to perform Increment operations on a single row. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java index 26bf4b4..4df496a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java @@ -21,15 +21,15 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScannable; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hbase.Cell; -import org.apache.hbase.CellScannable; -import org.apache.hbase.CellScanner; -import org.apache.hbase.CellUtil; import java.util.ArrayList; import java.util.HashMap; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java index 04305c9..7819723 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java @@ -28,13 +28,13 @@ import java.util.TreeMap; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hbase.Cell; /** * Used to perform Put operations for a single row. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java index 2e42c73..22170c4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java @@ -21,13 +21,13 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScannable; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.CellUtil; 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; -import org.apache.hbase.CellScannable; -import org.apache.hbase.CellScanner; -import org.apache.hbase.CellUtil; import java.nio.BufferOverflowException; import java.nio.ByteBuffer; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index c1efb2e..302489f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -26,6 +26,8 @@ import com.google.protobuf.Message; import com.google.protobuf.RpcChannel; import com.google.protobuf.Service; import com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -104,7 +106,6 @@ import org.apache.hadoop.hbase.util.Methods; 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 java.io.ByteArrayOutputStream; import java.io.IOException; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java new file mode 100644 index 0000000..036b4ed --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + + +/** + * The unit of storage in HBase consisting of the following fields:
+ *
+ * 1) row
+ * 2) column family
+ * 3) column qualifier
+ * 4) timestamp
+ * 5) type
+ * 6) MVCC version
+ * 7) value
+ * 
+ *

+ * Uniqueness is determined by the combination of row, column family, column qualifier, + * timestamp, and type. + *

+ * The natural comparator will perform a bitwise comparison on row, column family, and column + * qualifier. Less intuitively, it will then treat the greater timestamp as the lesser value with + * the goal of sorting newer cells first. + *

+ * This interface does not include methods that allocate new byte[]'s such as those used in client + * or debugging code. These should be placed in a sub-interface or the {@link CellUtil} class. + *

+ * Cell implements Comparable which is only meaningful when comparing to other keys in the + * same table. It uses CellComparator which does not work on the -ROOT- and .META. tables. + *

+ * In the future, we may consider adding a boolean isOnHeap() method and a getValueBuffer() method + * that can be used to pass a value directly from an off-heap ByteBuffer to the network without + * copying into an on-heap byte[]. + *

+ * Historic note: the original Cell implementation (KeyValue) requires that all fields be encoded as + * consecutive bytes in the same byte[], whereas this interface allows fields to reside in separate + * byte[]'s. + *

+ */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface Cell { + + //1) Row + + /** + * Contiguous raw bytes that may start at any index in the containing array. Max length is + * Short.MAX_VALUE which is 32,767 bytes. + * @return The array containing the row bytes. + */ + byte[] getRowArray(); + + /** + * @return Array index of first row byte + */ + int getRowOffset(); + + /** + * @return Number of row bytes. Must be < rowArray.length - offset. + */ + short getRowLength(); + + + //2) Family + + /** + * Contiguous bytes composed of legal HDFS filename characters which may start at any index in the + * containing array. Max length is Byte.MAX_VALUE, which is 127 bytes. + * @return the array containing the family bytes. + */ + byte[] getFamilyArray(); + + /** + * @return Array index of first row byte + */ + int getFamilyOffset(); + + /** + * @return Number of family bytes. Must be < familyArray.length - offset. + */ + byte getFamilyLength(); + + + //3) Qualifier + + /** + * Contiguous raw bytes that may start at any index in the containing array. Max length is + * Short.MAX_VALUE which is 32,767 bytes. + * @return The array containing the qualifier bytes. + */ + byte[] getQualifierArray(); + + /** + * @return Array index of first qualifier byte + */ + int getQualifierOffset(); + + /** + * @return Number of qualifier bytes. Must be < qualifierArray.length - offset. + */ + int getQualifierLength(); + + + //4) Timestamp + + /** + * @return Long value representing time at which this cell was "Put" into the row. Typically + * represents the time of insertion, but can be any value from Long.MIN_VALUE to Long.MAX_VALUE. + */ + long getTimestamp(); + + + //5) Type + + /** + * @return The byte representation of the KeyValue.TYPE of this cell: one of Put, Delete, etc + */ + byte getTypeByte(); + + + //6) MvccVersion + + /** + * Internal use only. A region-specific sequence ID given to each operation. It always exists for + * cells in the memstore but is not retained forever. It may survive several flushes, but + * generally becomes irrelevant after the cell's row is no longer involved in any operations that + * require strict consistency. + * @return mvccVersion (always >= 0 if exists), or 0 if it no longer exists + */ + long getMvccVersion(); + + + //7) Value + + /** + * Contiguous raw bytes that may start at any index in the containing array. Max length is + * Integer.MAX_VALUE which is 2,147,483,648 bytes. + * @return The array containing the value bytes. + */ + byte[] getValueArray(); + + /** + * @return Array index of first value byte + */ + int getValueOffset(); + + /** + * @return Number of value bytes. Must be < valueArray.length - offset. + */ + int getValueLength(); + +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java new file mode 100644 index 0000000..2e9e4f2 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import java.io.Serializable; +import java.util.Comparator; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; + +import com.google.common.primitives.Longs; + +/** + * Compare two HBase cells. Do not use this method comparing -ROOT- or + * .META. cells. Cells from these tables need a specialized comparator, one that + * takes account of the special formatting of the row where we have commas to delimit table from + * regionname, from row. See KeyValue for how it has a special comparator to do .META. cells + * and yet another for -ROOT-. + */ +@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){ + return equalsRow(a, b) + && equalsFamily(a, b) + && equalsQualifier(a, b) + && equalsTimestamp(a, b) + && equalsType(a, b); + } + + public static boolean equalsRow(Cell a, Cell b){ + return Bytes.equals( + a.getRowArray(), a.getRowOffset(), a.getRowLength(), + b.getRowArray(), b.getRowOffset(), b.getRowLength()); + } + + public static boolean equalsFamily(Cell a, Cell b){ + return Bytes.equals( + a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(), + b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength()); + } + + public static boolean equalsQualifier(Cell a, Cell b){ + return Bytes.equals( + a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(), + b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength()); + } + + public static boolean equalsTimestamp(Cell a, Cell b){ + return a.getTimestamp() == b.getTimestamp(); + } + + public static boolean equalsType(Cell a, Cell b){ + return a.getTypeByte() == b.getTypeByte(); + } + + + /********************* 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/hadoop/hbase/CellScannable.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScannable.java new file mode 100644 index 0000000..c2b895e --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScannable.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase; + +/** + * Implementer can return a CellScanner over its Cell content. + * Class name is ugly but mimicing java.util.Iterable only we are about the dumber + * CellScanner rather than say Iterator. See CellScanner class comment for why we go + * dumber than java.util.Iterator. + */ +public interface CellScannable { + /** + * @return A CellScanner over the contained {@link Cell}s + */ + CellScanner cellScanner(); +} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScanner.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScanner.java new file mode 100644 index 0000000..596710f --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScanner.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.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 #current()} 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. + *

+ * Typical usage: + * + *

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

Often used reading {@link org.apache.hadoop.hbase.Cell}s written by + * {@link org.apache.hadoop.hbase.io.CellOutputStream}. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface CellScanner { + /** + * @return the current Cell which may be mutable + */ + Cell current(); + + /** + * Advance the scanner 1 cell. + * @return true if the next cell is found and {@link #current()} will return a valid Cell + */ + boolean advance(); +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java new file mode 100644 index 0000000..d65aaab --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import java.util.Map.Entry; +import java.util.NavigableMap; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.ByteRange; + +/** + * Utility methods helpful slinging {@link Cell} instances. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public final class CellUtil { + + /******************* 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); + } + + /** + * @param cellScannerables + * @return CellScanner interface over cellIterables + */ + public static CellScanner createCellScanner(final List cellScannerables) { + return new CellScanner() { + private final Iterator iterator = cellScannerables.iterator(); + private CellScanner cellScanner = null; + + @Override + public Cell current() { + return this.cellScanner != null? this.cellScanner.current(): null; + } + + @Override + public boolean advance() { + if (this.cellScanner == null) { + if (!this.iterator.hasNext()) return false; + this.cellScanner = this.iterator.next().cellScanner(); + } + if (this.cellScanner.advance()) return true; + this.cellScanner = null; + return advance(); + } + }; + } + + /** + * @param cellIterable + * @return CellScanner interface over cellIterable + */ + public static CellScanner createCellScanner(final Iterable cellIterable) { + return createCellScanner(cellIterable.iterator()); + } + + /** + * @param cells + * @return CellScanner interface over cellIterable + */ + public static CellScanner createCellScanner(final Iterator cells) { + return new CellScanner() { + private final Iterator iterator = cells; + private Cell current = null; + + @Override + public Cell current() { + return this.current; + } + + @Override + public boolean advance() { + boolean hasNext = this.iterator.hasNext(); + this.current = hasNext? this.iterator.next(): null; + return hasNext; + } + }; + } + + /** + * @param cellArray + * @return CellScanner interface over cellArray + */ + public static CellScanner createCellScanner(final Cell[] cellArray) { + return new CellScanner() { + private final Cell [] cells = cellArray; + private int index = -1; + + @Override + public Cell current() { + return (index < 0)? null: this.cells[index]; + } + + @Override + public boolean advance() { + return ++index < this.cells.length; + } + }; + } + + /** + * Flatten the map of cells out under the CellScanner + * @param map Map of Cell Lists; for example, the map of families to Cells that is used + * inside Put, etc., keeping Cells organized by family. + * @return CellScanner interface over cellIterable + */ + public static CellScanner createCellScanner(final NavigableMap> map) { + return new CellScanner() { + private final Iterator>> entries = + map.entrySet().iterator(); + private Iterator currentIterator = null; + private Cell currentCell; + + @Override + public Cell current() { + return this.currentCell; + } + + @Override + public boolean advance() { + if (this.currentIterator == null) { + if (!this.entries.hasNext()) return false; + this.currentIterator = this.entries.next().getValue().iterator(); + } + if (this.currentIterator.hasNext()) { + this.currentCell = this.currentIterator.next(); + return true; + } + this.currentCell = null; + this.currentIterator = null; + return advance(); + } + }; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java index ca921ec..4dc2dae 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java @@ -39,8 +39,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.RawComparator; -import org.apache.hbase.Cell; -import org.apache.hbase.CellComparator; import com.google.common.primitives.Longs; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java index 8318522..1c20c06 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java @@ -26,7 +26,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.IterableUtils; import org.apache.hadoop.hbase.util.Strings; -import org.apache.hbase.CellComparator; import com.google.common.collect.Lists; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java index a865eca..10db743 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java @@ -26,8 +26,6 @@ import org.apache.hadoop.hbase.util.ByteRange; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.IterableUtils; import org.apache.hadoop.io.WritableUtils; -import org.apache.hbase.Cell; -import org.apache.hbase.CellUtil; /** * static convenience methods for dealing with KeyValues and collections of KeyValues diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java new file mode 100644 index 0000000..3743069 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.codec; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.hadoop.hbase.Cell; + +abstract class BaseDecoder implements Codec.Decoder { + final InputStream in; + private boolean hasNext = true; + private Cell current = null; + + BaseDecoder(final InputStream in) { + this.in = in; + } + + @Override + public boolean advance() { + if (!this.hasNext) return this.hasNext; + try { + if (this.in.available() <= 0) { + this.hasNext = false; + return this.hasNext; + } + this.current = parseCell(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return this.hasNext; + } + + /** + * @return extract a Cell + * @throws IOException + */ + abstract Cell parseCell() throws IOException; + + @Override + public Cell current() { + return this.current; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseEncoder.java new file mode 100644 index 0000000..c7a4aab --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseEncoder.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.codec; + +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.hadoop.hbase.Cell; + +abstract class BaseEncoder implements Codec.Encoder { + protected final OutputStream out; + // This encoder is 'done' once flush has been called. + protected boolean flushed = false; + + public BaseEncoder(final OutputStream out) { + this.out = out; + } + + @Override + public abstract void write(Cell cell) throws IOException; + + void checkFlushed() throws CodecException { + if (this.flushed) throw new CodecException("Flushed; done"); + } + + @Override + public void flush() throws IOException { + if (this.flushed) return; + this.flushed = true; + 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/hadoop/hbase/codec/CellCodec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java new file mode 100644 index 0000000..e3b7972 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.codec; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Basic Cell codec that just writes out all the individual elements of a Cell. Uses ints + * delimiting all lengths. Profligate. Needs tune up. Does not write the mvcc stamp. + * Use a different codec if you want that in the stream. + */ +public class CellCodec implements Codec { + static class CellEncoder extends BaseEncoder { + CellEncoder(final OutputStream out) { + super(out); + } + + @Override + public void write(Cell cell) throws IOException { + checkFlushed(); + try { + // Row + write(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); + // Column family + write(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + // Qualifier + write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + // Version + this.out.write(Bytes.toBytes(cell.getTimestamp())); + // Type + this.out.write(cell.getTypeByte()); + // Value + write(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); + } catch (IOException e) { + throw new CodecException(e); + } + } + + /** + * Write int length followed by array bytes. + * @param bytes + * @param offset + * @param length + * @throws IOException + */ + private void write(final byte [] bytes, final int offset, final int length) + throws IOException { + this.out.write(Bytes.toBytes(length)); + this.out.write(bytes, offset, length); + } + } + + static class CellDecoder extends BaseDecoder { + public CellDecoder(final InputStream in) { + super(in); + } + + Cell parseCell() throws IOException { + byte [] row = readByteArray(this.in); + byte [] family = readByteArray(in); + byte [] qualifier = readByteArray(in); + byte [] longArray = new byte[Bytes.SIZEOF_LONG]; + IOUtils.readFully(this.in, longArray); + long timestamp = Bytes.toLong(longArray); + byte type = (byte) this.in.read(); + byte [] value = readByteArray(in); + return CellUtil.createCell(row, family, qualifier, timestamp, type, value); + } + + /** + * @return Byte array read from the stream. + * @throws IOException + */ + private byte [] readByteArray(final InputStream in) throws IOException { + byte [] intArray = new byte[Bytes.SIZEOF_INT]; + IOUtils.readFully(in, intArray); + int length = Bytes.toInt(intArray); + byte [] bytes = new byte [length]; + IOUtils.readFully(in, bytes); + return bytes; + } + } + + @Override + public Decoder getDecoder(InputStream is) { + return new CellDecoder(is); + } + + @Override + public Encoder getEncoder(OutputStream os) { + return new CellEncoder(os); + } +} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java new file mode 100644 index 0000000..a89cc2b --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.codec; + +import java.io.InputStream; +import java.io.OutputStream; + +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.io.CellOutputStream; +import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder; + +/** + * Encoder/Decoder for Cell. + * + *

Like {@link DataBlockEncoder} only Cell-based rather than KeyValue version 1 based + * and without presuming an hfile context. Intent is an Interface that will work for hfile and + * rpc. + */ +public interface Codec { + // TODO: interfacing with {@link DataBlockEncoder} + /** + * Call flush when done. Some encoders may not put anything on the stream until flush is called. + * On flush, let go of any resources used by the encoder. + */ + public interface Encoder extends CellOutputStream {} + + /** + * Implementations should implicitly clean up any resources allocated when the + * Decoder/CellScanner runs off the end of the cell block. Do this rather than require the user + * call close explicitly. + */ + public interface Decoder extends CellScanner {}; + + Decoder getDecoder(InputStream is); + Encoder getEncoder(OutputStream os); +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CodecException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CodecException.java new file mode 100644 index 0000000..15919cd --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CodecException.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.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/hadoop/hbase/codec/KeyValueCodec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java new file mode 100644 index 0000000..33a21b8 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.codec; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; + +/** + * Codec that does KeyValue version 1 serialization. + * + *

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
+ * 
+ */ +public class KeyValueCodec implements Codec { + static class KeyValueEncoder extends BaseEncoder { + KeyValueEncoder(final OutputStream out) { + super(out); + } + + @Override + public void write(Cell cell) throws IOException { + checkFlushed(); + // This is crass and will not work when KV changes. Also if passed a non-kv Cell, it will + // make expensive copy. + try { + KeyValue.oswrite((KeyValue)KeyValueUtil.ensureKeyValue(cell), this.out); + } catch (IOException e) { + throw new CodecException(e); + } + } + } + + static class KeyValueDecoder extends BaseDecoder { + KeyValueDecoder(final InputStream in) { + super(in); + } + + Cell parseCell() throws IOException { + return KeyValue.iscreate(in); + } + } + + /** + * Implementation depends on {@link InputStream#available()} + */ + @Override + public Decoder getDecoder(final InputStream is) { + return new KeyValueDecoder(is); + } + + @Override + public Encoder getEncoder(OutputStream os) { + return new KeyValueEncoder(os); + } +} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/CellOutputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/CellOutputStream.java new file mode 100644 index 0000000..05b3403 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/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.hadoop.hbase.io; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.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/main/java/org/apache/hbase/Cell.java b/hbase-common/src/main/java/org/apache/hbase/Cell.java deleted file mode 100644 index 3dc7941..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/Cell.java +++ /dev/null @@ -1,171 +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; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - - -/** - * The unit of storage in HBase consisting of the following fields:
- *

- * 1) row
- * 2) column family
- * 3) column qualifier
- * 4) timestamp
- * 5) type
- * 6) MVCC version
- * 7) value
- * 
- *

- * Uniqueness is determined by the combination of row, column family, column qualifier, - * timestamp, and type. - *

- * The natural comparator will perform a bitwise comparison on row, column family, and column - * qualifier. Less intuitively, it will then treat the greater timestamp as the lesser value with - * the goal of sorting newer cells first. - *

- * This interface does not include methods that allocate new byte[]'s such as those used in client - * or debugging code. These should be placed in a sub-interface or the {@link CellUtil} class. - *

- * Cell implements Comparable which is only meaningful when comparing to other keys in the - * same table. It uses CellComparator which does not work on the -ROOT- and .META. tables. - *

- * In the future, we may consider adding a boolean isOnHeap() method and a getValueBuffer() method - * that can be used to pass a value directly from an off-heap ByteBuffer to the network without - * copying into an on-heap byte[]. - *

- * Historic note: the original Cell implementation (KeyValue) requires that all fields be encoded as - * consecutive bytes in the same byte[], whereas this interface allows fields to reside in separate - * byte[]'s. - *

- */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -public interface Cell { - - //1) Row - - /** - * Contiguous raw bytes that may start at any index in the containing array. Max length is - * Short.MAX_VALUE which is 32,767 bytes. - * @return The array containing the row bytes. - */ - byte[] getRowArray(); - - /** - * @return Array index of first row byte - */ - int getRowOffset(); - - /** - * @return Number of row bytes. Must be < rowArray.length - offset. - */ - short getRowLength(); - - - //2) Family - - /** - * Contiguous bytes composed of legal HDFS filename characters which may start at any index in the - * containing array. Max length is Byte.MAX_VALUE, which is 127 bytes. - * @return the array containing the family bytes. - */ - byte[] getFamilyArray(); - - /** - * @return Array index of first row byte - */ - int getFamilyOffset(); - - /** - * @return Number of family bytes. Must be < familyArray.length - offset. - */ - byte getFamilyLength(); - - - //3) Qualifier - - /** - * Contiguous raw bytes that may start at any index in the containing array. Max length is - * Short.MAX_VALUE which is 32,767 bytes. - * @return The array containing the qualifier bytes. - */ - byte[] getQualifierArray(); - - /** - * @return Array index of first qualifier byte - */ - int getQualifierOffset(); - - /** - * @return Number of qualifier bytes. Must be < qualifierArray.length - offset. - */ - int getQualifierLength(); - - - //4) Timestamp - - /** - * @return Long value representing time at which this cell was "Put" into the row. Typically - * represents the time of insertion, but can be any value from Long.MIN_VALUE to Long.MAX_VALUE. - */ - long getTimestamp(); - - - //5) Type - - /** - * @return The byte representation of the KeyValue.TYPE of this cell: one of Put, Delete, etc - */ - byte getTypeByte(); - - - //6) MvccVersion - - /** - * Internal use only. A region-specific sequence ID given to each operation. It always exists for - * cells in the memstore but is not retained forever. It may survive several flushes, but - * generally becomes irrelevant after the cell's row is no longer involved in any operations that - * require strict consistency. - * @return mvccVersion (always >= 0 if exists), or 0 if it no longer exists - */ - long getMvccVersion(); - - - //7) Value - - /** - * Contiguous raw bytes that may start at any index in the containing array. Max length is - * Integer.MAX_VALUE which is 2,147,483,648 bytes. - * @return The array containing the value bytes. - */ - byte[] getValueArray(); - - /** - * @return Array index of first value byte - */ - int getValueOffset(); - - /** - * @return Number of value bytes. Must be < valueArray.length - offset. - */ - int getValueLength(); - -} diff --git a/hbase-common/src/main/java/org/apache/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hbase/CellComparator.java deleted file mode 100644 index 81aaa95..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/CellComparator.java +++ /dev/null @@ -1,198 +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; - -import java.io.Serializable; -import java.util.Comparator; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.hbase.util.Bytes; - -import com.google.common.primitives.Longs; - -/** - * Compare two HBase cells. Do not use this method comparing -ROOT- or - * .META. cells. Cells from these tables need a specialized comparator, one that - * takes account of the special formatting of the row where we have commas to delimit table from - * regionname, from row. See KeyValue for how it has a special comparator to do .META. cells - * and yet another for -ROOT-. - */ -@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){ - return equalsRow(a, b) - && equalsFamily(a, b) - && equalsQualifier(a, b) - && equalsTimestamp(a, b) - && equalsType(a, b); - } - - public static boolean equalsRow(Cell a, Cell b){ - return Bytes.equals( - a.getRowArray(), a.getRowOffset(), a.getRowLength(), - b.getRowArray(), b.getRowOffset(), b.getRowLength()); - } - - public static boolean equalsFamily(Cell a, Cell b){ - return Bytes.equals( - a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(), - b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength()); - } - - public static boolean equalsQualifier(Cell a, Cell b){ - return Bytes.equals( - a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(), - b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength()); - } - - public static boolean equalsTimestamp(Cell a, Cell b){ - return a.getTimestamp() == b.getTimestamp(); - } - - public static boolean equalsType(Cell a, Cell b){ - return a.getTypeByte() == b.getTypeByte(); - } - - - /********************* 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/CellScannable.java b/hbase-common/src/main/java/org/apache/hbase/CellScannable.java deleted file mode 100644 index deefaab..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/CellScannable.java +++ /dev/null @@ -1,31 +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; - -/** - * Implementer can return a CellScanner over its Cell content. - * Class name is ugly but mimicing java.util.Iterable only we are about the dumber - * CellScanner rather than say Iterator. See CellScanner class comment for why we go - * dumber than java.util.Iterator. - */ -public interface CellScannable { - /** - * @return A CellScanner over the contained {@link Cell}s - */ - CellScanner cellScanner(); -} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hbase/CellScanner.java b/hbase-common/src/main/java/org/apache/hbase/CellScanner.java deleted file mode 100644 index 34a5e21..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/CellScanner.java +++ /dev/null @@ -1,62 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 #current()} 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. - *

- * Typical usage: - * - *

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

Often used reading {@link org.apache.hbase.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 current(); - - /** - * Advance the scanner 1 cell. - * @return true if the next cell is found and {@link #current()} will return a valid Cell - */ - boolean advance(); -} diff --git a/hbase-common/src/main/java/org/apache/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hbase/CellUtil.java deleted file mode 100644 index b496035..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/CellUtil.java +++ /dev/null @@ -1,245 +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; - -import java.nio.ByteBuffer; -import java.util.Iterator; -import java.util.List; -import java.util.Map.Entry; -import java.util.NavigableMap; - -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; - -/** - * Utility methods helpful slinging {@link Cell} instances. - */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -public final class CellUtil { - - /******************* 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); - } - - /** - * @param cellScannerables - * @return CellScanner interface over cellIterables - */ - public static CellScanner createCellScanner(final List cellScannerables) { - return new CellScanner() { - private final Iterator iterator = cellScannerables.iterator(); - private CellScanner cellScanner = null; - - @Override - public Cell current() { - return this.cellScanner != null? this.cellScanner.current(): null; - } - - @Override - public boolean advance() { - if (this.cellScanner == null) { - if (!this.iterator.hasNext()) return false; - this.cellScanner = this.iterator.next().cellScanner(); - } - if (this.cellScanner.advance()) return true; - this.cellScanner = null; - return advance(); - } - }; - } - - /** - * @param cellIterable - * @return CellScanner interface over cellIterable - */ - public static CellScanner createCellScanner(final Iterable cellIterable) { - return createCellScanner(cellIterable.iterator()); - } - - /** - * @param cells - * @return CellScanner interface over cellIterable - */ - public static CellScanner createCellScanner(final Iterator cells) { - return new CellScanner() { - private final Iterator iterator = cells; - private Cell current = null; - - @Override - public Cell current() { - return this.current; - } - - @Override - public boolean advance() { - boolean hasNext = this.iterator.hasNext(); - this.current = hasNext? this.iterator.next(): null; - return hasNext; - } - }; - } - - /** - * @param cellArray - * @return CellScanner interface over cellArray - */ - public static CellScanner createCellScanner(final Cell[] cellArray) { - return new CellScanner() { - private final Cell [] cells = cellArray; - private int index = -1; - - @Override - public Cell current() { - return (index < 0)? null: this.cells[index]; - } - - @Override - public boolean advance() { - return ++index < this.cells.length; - } - }; - } - - /** - * Flatten the map of cells out under the CellScanner - * @param map Map of Cell Lists; for example, the map of families to Cells that is used - * inside Put, etc., keeping Cells organized by family. - * @return CellScanner interface over cellIterable - */ - public static CellScanner createCellScanner(final NavigableMap> map) { - return new CellScanner() { - private final Iterator>> entries = - map.entrySet().iterator(); - private Iterator currentIterator = null; - private Cell currentCell; - - @Override - public Cell current() { - return this.currentCell; - } - - @Override - public boolean advance() { - if (this.currentIterator == null) { - if (!this.entries.hasNext()) return false; - this.currentIterator = this.entries.next().getValue().iterator(); - } - if (this.currentIterator.hasNext()) { - this.currentCell = this.currentIterator.next(); - return true; - } - this.currentCell = null; - this.currentIterator = null; - return advance(); - } - }; - } -} diff --git a/hbase-common/src/main/java/org/apache/hbase/codec/BaseDecoder.java b/hbase-common/src/main/java/org/apache/hbase/codec/BaseDecoder.java deleted file mode 100644 index 0c941df..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/codec/BaseDecoder.java +++ /dev/null @@ -1,59 +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.codec; - -import java.io.IOException; -import java.io.InputStream; - -import org.apache.hbase.Cell; - -abstract class BaseDecoder implements Codec.Decoder { - final InputStream in; - private boolean hasNext = true; - private Cell current = null; - - BaseDecoder(final InputStream in) { - this.in = in; - } - - @Override - public boolean advance() { - if (!this.hasNext) return this.hasNext; - try { - if (this.in.available() <= 0) { - this.hasNext = false; - return this.hasNext; - } - this.current = parseCell(); - } catch (IOException e) { - throw new RuntimeException(e); - } - return this.hasNext; - } - - /** - * @return extract a Cell - * @throws IOException - */ - abstract Cell parseCell() throws IOException; - - @Override - public Cell current() { - return this.current; - } -} diff --git a/hbase-common/src/main/java/org/apache/hbase/codec/BaseEncoder.java b/hbase-common/src/main/java/org/apache/hbase/codec/BaseEncoder.java deleted file mode 100644 index 8a2bb2e..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/codec/BaseEncoder.java +++ /dev/null @@ -1,51 +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.codec; - -import java.io.IOException; -import java.io.OutputStream; - -import org.apache.hbase.Cell; - -abstract class BaseEncoder implements Codec.Encoder { - protected final OutputStream out; - // This encoder is 'done' once flush has been called. - protected boolean flushed = false; - - public BaseEncoder(final OutputStream out) { - this.out = out; - } - - @Override - public abstract void write(Cell cell) throws IOException; - - void checkFlushed() throws CodecException { - if (this.flushed) throw new CodecException("Flushed; done"); - } - - @Override - public void flush() throws IOException { - if (this.flushed) return; - this.flushed = true; - 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/codec/CellCodec.java b/hbase-common/src/main/java/org/apache/hbase/codec/CellCodec.java deleted file mode 100644 index 5c1dd83..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/codec/CellCodec.java +++ /dev/null @@ -1,115 +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.codec; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; - -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; -import org.apache.hbase.CellUtil; - -/** - * Basic Cell codec that just writes out all the individual elements of a Cell. Uses ints - * delimiting all lengths. Profligate. Needs tune up. Does not write the mvcc stamp. - * Use a different codec if you want that in the stream. - */ -public class CellCodec implements Codec { - static class CellEncoder extends BaseEncoder { - CellEncoder(final OutputStream out) { - super(out); - } - - @Override - public void write(Cell cell) throws IOException { - checkFlushed(); - try { - // Row - write(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); - // Column family - write(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); - // Qualifier - write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); - // Version - this.out.write(Bytes.toBytes(cell.getTimestamp())); - // Type - this.out.write(cell.getTypeByte()); - // Value - write(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); - } catch (IOException e) { - throw new CodecException(e); - } - } - - /** - * Write int length followed by array bytes. - * @param bytes - * @param offset - * @param length - * @throws IOException - */ - private void write(final byte [] bytes, final int offset, final int length) - throws IOException { - this.out.write(Bytes.toBytes(length)); - this.out.write(bytes, offset, length); - } - } - - static class CellDecoder extends BaseDecoder { - public CellDecoder(final InputStream in) { - super(in); - } - - Cell parseCell() throws IOException { - byte [] row = readByteArray(this.in); - byte [] family = readByteArray(in); - byte [] qualifier = readByteArray(in); - byte [] longArray = new byte[Bytes.SIZEOF_LONG]; - IOUtils.readFully(this.in, longArray); - long timestamp = Bytes.toLong(longArray); - byte type = (byte) this.in.read(); - byte [] value = readByteArray(in); - return CellUtil.createCell(row, family, qualifier, timestamp, type, value); - } - - /** - * @return Byte array read from the stream. - * @throws IOException - */ - private byte [] readByteArray(final InputStream in) throws IOException { - byte [] intArray = new byte[Bytes.SIZEOF_INT]; - IOUtils.readFully(in, intArray); - int length = Bytes.toInt(intArray); - byte [] bytes = new byte [length]; - IOUtils.readFully(in, bytes); - return bytes; - } - } - - @Override - public Decoder getDecoder(InputStream is) { - return new CellDecoder(is); - } - - @Override - public Encoder getEncoder(OutputStream os) { - return new CellEncoder(os); - } -} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hbase/codec/Codec.java b/hbase-common/src/main/java/org/apache/hbase/codec/Codec.java deleted file mode 100644 index 4312744..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/codec/Codec.java +++ /dev/null @@ -1,51 +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.codec; - -import java.io.InputStream; -import java.io.OutputStream; - -import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder; -import org.apache.hbase.CellScanner; -import org.apache.hbase.io.CellOutputStream; - -/** - * Encoder/Decoder for Cell. - * - *

Like {@link DataBlockEncoder} only Cell-based rather than KeyValue version 1 based - * and without presuming an hfile context. Intent is an Interface that will work for hfile and - * rpc. - */ -public interface Codec { - // TODO: interfacing with {@link DataBlockEncoder} - /** - * Call flush when done. Some encoders may not put anything on the stream until flush is called. - * On flush, let go of any resources used by the encoder. - */ - public interface Encoder extends CellOutputStream {} - - /** - * Implementations should implicitly clean up any resources allocated when the - * Decoder/CellScanner runs off the end of the cell block. Do this rather than require the user - * call close explicitly. - */ - public interface Decoder extends CellScanner {}; - - Decoder getDecoder(InputStream is); - Encoder getEncoder(OutputStream os); -} 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 deleted file mode 100644 index 352e1a6..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/codec/CodecException.java +++ /dev/null @@ -1,39 +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.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/KeyValueCodec.java b/hbase-common/src/main/java/org/apache/hbase/codec/KeyValueCodec.java deleted file mode 100644 index da06973..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/codec/KeyValueCodec.java +++ /dev/null @@ -1,86 +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.codec; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hbase.Cell; - -/** - * Codec that does KeyValue version 1 serialization. - * - *

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
- * 
- */ -public class KeyValueCodec implements Codec { - static class KeyValueEncoder extends BaseEncoder { - KeyValueEncoder(final OutputStream out) { - super(out); - } - - @Override - public void write(Cell cell) throws IOException { - checkFlushed(); - // This is crass and will not work when KV changes. Also if passed a non-kv Cell, it will - // make expensive copy. - try { - KeyValue.oswrite((KeyValue)KeyValueUtil.ensureKeyValue(cell), this.out); - } catch (IOException e) { - throw new CodecException(e); - } - } - } - - static class KeyValueDecoder extends BaseDecoder { - KeyValueDecoder(final InputStream in) { - super(in); - } - - Cell parseCell() throws IOException { - return KeyValue.iscreate(in); - } - } - - /** - * Implementation depends on {@link InputStream#available()} - */ - @Override - public Decoder getDecoder(final InputStream is) { - return new KeyValueDecoder(is); - } - - @Override - public Encoder getEncoder(OutputStream os) { - return new KeyValueEncoder(os); - } -} \ 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 deleted file mode 100644 index 1c94622..0000000 --- a/hbase-common/src/main/java/org/apache/hbase/io/CellOutputStream.java +++ /dev/null @@ -1,55 +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.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/hadoop/hbase/TestCellUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java new file mode 100644 index 0000000..3f10259 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase; + +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.List; +import java.util.NavigableMap; +import java.util.TreeMap; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestCellUtil { + @Test + public void testCreateCellScannerCellList() { + final int count = 3; + Cell [] cs = getCells(count, Bytes.toBytes(0)); + List cells = Arrays.asList(cs); + CellScanner scanner = CellUtil.createCellScanner(cells); + int i = 0; + while (scanner.advance()) { + i++; + } + assertEquals(count, i); + } + + @Test + public void testCreateCellScannerFamilyMap() { + final int count = 3; + final NavigableMap> map = + new TreeMap>(Bytes.BYTES_COMPARATOR); + for (int i = 0; i < count; i++) { + byte [] key = Bytes.toBytes(i); + KeyValue [] cs = getCells(count, key); + map.put(key, Arrays.asList(cs)); + } + CellScanner scanner = CellUtil.createCellScanner(map); + int i = 0; + while (scanner.advance()) { + i++; + } + assertEquals(count * count, i); + } + + static KeyValue [] getCells(final int howMany, final byte [] family) { + KeyValue [] cells = new KeyValue[howMany]; + for (int i = 0; i < howMany; i++) { + byte [] index = Bytes.toBytes(i); + KeyValue kv = new KeyValue(index, family, index, index); + cells[i] = kv; + } + return cells; + } +} \ No newline at end of file diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java new file mode 100644 index 0000000..1bb990e --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java @@ -0,0 +1,124 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.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.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.codec.CellCodec; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.io.CountingInputStream; +import com.google.common.io.CountingOutputStream; + +@Category(SmallTests.class) +public class TestCellCodec { + + @Test + public void testEmptyWorks() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + CountingOutputStream cos = new CountingOutputStream(baos); + DataOutputStream dos = new DataOutputStream(cos); + Codec codec = new CellCodec(); + Codec.Encoder encoder = codec.getEncoder(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); + Codec.Decoder decoder = codec.getDecoder(dis); + assertFalse(decoder.advance()); + 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); + Codec codec = new CellCodec(); + Codec.Encoder encoder = codec.getEncoder(dos); + final KeyValue kv = + new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"), Bytes.toBytes("v")); + encoder.write(kv); + encoder.flush(); + dos.close(); + long offset = cos.getCount(); + CountingInputStream cis = + new CountingInputStream(new ByteArrayInputStream(baos.toByteArray())); + DataInputStream dis = new DataInputStream(cis); + Codec.Decoder decoder = codec.getDecoder(dis); + assertTrue(decoder.advance()); // First read should pull in the KV + // Second read should trip over the end-of-stream marker and return false + assertFalse(decoder.advance()); + dis.close(); + assertEquals(offset, cis.getCount()); + } + + @Test + public void testThree() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + CountingOutputStream cos = new CountingOutputStream(baos); + DataOutputStream dos = new DataOutputStream(cos); + Codec codec = new CellCodec(); + Codec.Encoder encoder = codec.getEncoder(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")); + encoder.write(kv1); + encoder.write(kv2); + encoder.write(kv3); + encoder.flush(); + dos.close(); + long offset = cos.getCount(); + CountingInputStream cis = + new CountingInputStream(new ByteArrayInputStream(baos.toByteArray())); + DataInputStream dis = new DataInputStream(cis); + Codec.Decoder decoder = codec.getDecoder(dis); + assertTrue(decoder.advance()); + Cell c = decoder.current(); + assertTrue(CellComparator.equals(c, kv1)); + assertTrue(decoder.advance()); + c = decoder.current(); + assertTrue(CellComparator.equals(c, kv2)); + assertTrue(decoder.advance()); + c = decoder.current(); + assertTrue(CellComparator.equals(c, kv3)); + assertFalse(decoder.advance()); + dis.close(); + assertEquals(offset, cis.getCount()); + } +} \ No newline at end of file diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java new file mode 100644 index 0000000..3d254af --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java @@ -0,0 +1,127 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.codec; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +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.SmallTests; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.codec.KeyValueCodec; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.io.CountingInputStream; +import com.google.common.io.CountingOutputStream; + +@Category(SmallTests.class) +public class TestKeyValueCodec { + @Test + public void testEmptyWorks() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + CountingOutputStream cos = new CountingOutputStream(baos); + DataOutputStream dos = new DataOutputStream(cos); + KeyValueCodec kvc = new KeyValueCodec(); + Codec.Encoder encoder = kvc.getEncoder(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); + Codec.Decoder decoder = kvc.getDecoder(dis); + assertFalse(decoder.advance()); + 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); + KeyValueCodec kvc = new KeyValueCodec(); + Codec.Encoder encoder = kvc.getEncoder(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); + Codec.Decoder decoder = kvc.getDecoder(dis); + assertTrue(decoder.advance()); // First read should pull in the KV + // Second read should trip over the end-of-stream marker and return false + assertFalse(decoder.advance()); + 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); + KeyValueCodec kvc = new KeyValueCodec(); + Codec.Encoder encoder = kvc.getEncoder(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); + Codec.Decoder decoder = kvc.getDecoder(dis); + assertTrue(decoder.advance()); + KeyValue kv = (KeyValue)decoder.current(); + assertTrue(kv1.equals(kv)); + assertTrue(decoder.advance()); + kv = (KeyValue)decoder.current(); + assertTrue(kv2.equals(kv)); + assertTrue(decoder.advance()); + kv = (KeyValue)decoder.current(); + assertTrue(kv3.equals(kv)); + assertFalse(decoder.advance()); + dis.close(); + assertEquals((length * 3), cis.getCount()); + } +} \ No newline at end of file diff --git a/hbase-common/src/test/java/org/apache/hbase/TestCellUtil.java b/hbase-common/src/test/java/org/apache/hbase/TestCellUtil.java deleted file mode 100644 index ca07720..0000000 --- a/hbase-common/src/test/java/org/apache/hbase/TestCellUtil.java +++ /dev/null @@ -1,72 +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; - -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.List; -import java.util.NavigableMap; -import java.util.TreeMap; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Test; - -public class TestCellUtil { - @Test - public void testCreateCellScannerCellList() { - final int count = 3; - Cell [] cs = getCells(count, Bytes.toBytes(0)); - List cells = Arrays.asList(cs); - CellScanner scanner = CellUtil.createCellScanner(cells); - int i = 0; - while (scanner.advance()) { - i++; - } - assertEquals(count, i); - } - - @Test - public void testCreateCellScannerFamilyMap() { - final int count = 3; - final NavigableMap> map = - new TreeMap>(Bytes.BYTES_COMPARATOR); - for (int i = 0; i < count; i++) { - byte [] key = Bytes.toBytes(i); - KeyValue [] cs = getCells(count, key); - map.put(key, Arrays.asList(cs)); - } - CellScanner scanner = CellUtil.createCellScanner(map); - int i = 0; - while (scanner.advance()) { - i++; - } - assertEquals(count * count, i); - } - - static KeyValue [] getCells(final int howMany, final byte [] family) { - KeyValue [] cells = new KeyValue[howMany]; - for (int i = 0; i < howMany; i++) { - byte [] index = Bytes.toBytes(i); - KeyValue kv = new KeyValue(index, family, index, index); - cells[i] = kv; - } - return cells; - } -} \ No newline at end of file diff --git a/hbase-common/src/test/java/org/apache/hbase/codec/TestCellCodec.java b/hbase-common/src/test/java/org/apache/hbase/codec/TestCellCodec.java deleted file mode 100644 index bb47099..0000000 --- a/hbase-common/src/test/java/org/apache/hbase/codec/TestCellCodec.java +++ /dev/null @@ -1,122 +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.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.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; -import org.apache.hbase.CellComparator; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import com.google.common.io.CountingInputStream; -import com.google.common.io.CountingOutputStream; - -@Category(SmallTests.class) -public class TestCellCodec { - - @Test - public void testEmptyWorks() throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - CountingOutputStream cos = new CountingOutputStream(baos); - DataOutputStream dos = new DataOutputStream(cos); - Codec codec = new CellCodec(); - Codec.Encoder encoder = codec.getEncoder(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); - Codec.Decoder decoder = codec.getDecoder(dis); - assertFalse(decoder.advance()); - 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); - Codec codec = new CellCodec(); - Codec.Encoder encoder = codec.getEncoder(dos); - final KeyValue kv = - new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"), Bytes.toBytes("v")); - encoder.write(kv); - encoder.flush(); - dos.close(); - long offset = cos.getCount(); - CountingInputStream cis = - new CountingInputStream(new ByteArrayInputStream(baos.toByteArray())); - DataInputStream dis = new DataInputStream(cis); - Codec.Decoder decoder = codec.getDecoder(dis); - assertTrue(decoder.advance()); // First read should pull in the KV - // Second read should trip over the end-of-stream marker and return false - assertFalse(decoder.advance()); - dis.close(); - assertEquals(offset, cis.getCount()); - } - - @Test - public void testThree() throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - CountingOutputStream cos = new CountingOutputStream(baos); - DataOutputStream dos = new DataOutputStream(cos); - Codec codec = new CellCodec(); - Codec.Encoder encoder = codec.getEncoder(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")); - encoder.write(kv1); - encoder.write(kv2); - encoder.write(kv3); - encoder.flush(); - dos.close(); - long offset = cos.getCount(); - CountingInputStream cis = - new CountingInputStream(new ByteArrayInputStream(baos.toByteArray())); - DataInputStream dis = new DataInputStream(cis); - Codec.Decoder decoder = codec.getDecoder(dis); - assertTrue(decoder.advance()); - Cell c = decoder.current(); - assertTrue(CellComparator.equals(c, kv1)); - assertTrue(decoder.advance()); - c = decoder.current(); - assertTrue(CellComparator.equals(c, kv2)); - assertTrue(decoder.advance()); - c = decoder.current(); - assertTrue(CellComparator.equals(c, kv3)); - assertFalse(decoder.advance()); - dis.close(); - assertEquals(offset, cis.getCount()); - } -} \ No newline at end of file diff --git a/hbase-common/src/test/java/org/apache/hbase/codec/TestKeyValueCodec.java b/hbase-common/src/test/java/org/apache/hbase/codec/TestKeyValueCodec.java deleted file mode 100644 index daa2b89..0000000 --- a/hbase-common/src/test/java/org/apache/hbase/codec/TestKeyValueCodec.java +++ /dev/null @@ -1,125 +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.codec; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -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.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import com.google.common.io.CountingInputStream; -import com.google.common.io.CountingOutputStream; - -@Category(SmallTests.class) -public class TestKeyValueCodec { - @Test - public void testEmptyWorks() throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - CountingOutputStream cos = new CountingOutputStream(baos); - DataOutputStream dos = new DataOutputStream(cos); - KeyValueCodec kvc = new KeyValueCodec(); - Codec.Encoder encoder = kvc.getEncoder(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); - Codec.Decoder decoder = kvc.getDecoder(dis); - assertFalse(decoder.advance()); - 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); - KeyValueCodec kvc = new KeyValueCodec(); - Codec.Encoder encoder = kvc.getEncoder(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); - Codec.Decoder decoder = kvc.getDecoder(dis); - assertTrue(decoder.advance()); // First read should pull in the KV - // Second read should trip over the end-of-stream marker and return false - assertFalse(decoder.advance()); - 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); - KeyValueCodec kvc = new KeyValueCodec(); - Codec.Encoder encoder = kvc.getEncoder(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); - Codec.Decoder decoder = kvc.getDecoder(dis); - assertTrue(decoder.advance()); - KeyValue kv = (KeyValue)decoder.current(); - assertTrue(kv1.equals(kv)); - assertTrue(decoder.advance()); - kv = (KeyValue)decoder.current(); - assertTrue(kv2.equals(kv)); - assertTrue(decoder.advance()); - kv = (KeyValue)decoder.current(); - assertTrue(kv3.equals(kv)); - assertFalse(decoder.advance()); - dis.close(); - assertEquals((length * 3), cis.getCount()); - } -} \ No newline at end of file diff --git a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/PrefixTreeSeeker.java b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/PrefixTreeSeeker.java index 70cd0fd..85c6484 100644 --- a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/PrefixTreeSeeker.java +++ b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/PrefixTreeSeeker.java @@ -21,11 +21,11 @@ package org.apache.hbase.codec.prefixtree; import java.nio.ByteBuffer; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder.EncodedSeeker; -import org.apache.hbase.Cell; -import org.apache.hbase.CellUtil; import org.apache.hbase.codec.prefixtree.decode.DecoderFactory; import org.apache.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher; import org.apache.hbase.codec.prefixtree.scanner.CellScannerPosition; diff --git a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java index 49b968f..398bd5d 100644 --- a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java +++ b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java @@ -19,9 +19,9 @@ package org.apache.hbase.codec.prefixtree.decode; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hbase.Cell; -import org.apache.hbase.CellComparator; -import org.apache.hbase.CellScanner; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellScanner; import org.apache.hbase.codec.prefixtree.PrefixTreeBlockMeta; import org.apache.hbase.codec.prefixtree.decode.column.ColumnReader; import org.apache.hbase.codec.prefixtree.decode.row.RowNodeReader; diff --git a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java index eab3c0b..5201b6d 100644 --- a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java +++ b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java @@ -19,8 +19,8 @@ package org.apache.hbase.codec.prefixtree.decode; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hbase.Cell; -import org.apache.hbase.CellUtil; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hbase.codec.prefixtree.PrefixTreeBlockMeta; import org.apache.hbase.codec.prefixtree.scanner.CellScannerPosition; import org.apache.hbase.codec.prefixtree.scanner.CellSearcher; diff --git a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeCell.java b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeCell.java index 040bf19..5573c02 100644 --- a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeCell.java +++ b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeCell.java @@ -19,10 +19,10 @@ package org.apache.hbase.codec.prefixtree.decode; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hbase.Cell; -import org.apache.hbase.CellComparator; /** * As the PrefixTreeArrayScanner moves through the tree bytes, it changes the values in the fields diff --git a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java index af91cd3..46cb707 100644 --- a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java +++ b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java @@ -24,19 +24,19 @@ import java.io.OutputStream; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.io.CellOutputStream; import org.apache.hadoop.hbase.util.ArrayUtils; import org.apache.hadoop.hbase.util.ByteRange; import org.apache.hadoop.io.WritableUtils; -import org.apache.hbase.Cell; -import org.apache.hbase.CellUtil; import org.apache.hbase.codec.prefixtree.PrefixTreeBlockMeta; import org.apache.hbase.codec.prefixtree.encode.column.ColumnSectionWriter; import org.apache.hbase.codec.prefixtree.encode.other.CellTypeEncoder; import org.apache.hbase.codec.prefixtree.encode.other.LongEncoder; import org.apache.hbase.codec.prefixtree.encode.row.RowSectionWriter; import org.apache.hbase.codec.prefixtree.encode.tokenize.Tokenizer; -import org.apache.hbase.io.CellOutputStream; import org.apache.hbase.util.byterange.ByteRangeSet; import org.apache.hbase.util.byterange.impl.ByteRangeHashSet; import org.apache.hbase.util.byterange.impl.ByteRangeTreeSet; diff --git a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/CellSearcher.java b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/CellSearcher.java index 77aca62..e55c559 100644 --- a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/CellSearcher.java +++ b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/CellSearcher.java @@ -19,7 +19,7 @@ package org.apache.hbase.codec.prefixtree.scanner; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hbase.Cell; +import org.apache.hadoop.hbase.Cell; /** * Methods for seeking to a random {@link Cell} inside a sorted collection of cells. Indicates that diff --git a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java index 5e236d7..b4463d8 100644 --- a/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java +++ b/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java @@ -19,7 +19,7 @@ package org.apache.hbase.codec.prefixtree.scanner; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hbase.CellScanner; +import org.apache.hadoop.hbase.CellScanner; /** * An extension of CellScanner indicating the scanner supports iterating backwards through cells. diff --git a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/BaseTestRowData.java b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/BaseTestRowData.java index 716b7f3..9c3fcf9 100644 --- a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/BaseTestRowData.java +++ b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/BaseTestRowData.java @@ -20,8 +20,8 @@ package org.apache.hbase.codec.prefixtree.row; import java.util.List; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hbase.CellComparator; import org.apache.hbase.codec.prefixtree.PrefixTreeBlockMeta; import org.apache.hbase.codec.prefixtree.scanner.CellSearcher; diff --git a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java index 9e05bf3..b138d1e 100644 --- a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java +++ b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java @@ -24,11 +24,11 @@ import java.nio.ByteBuffer; import java.util.Collection; import java.util.List; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.util.CollectionUtils; -import org.apache.hbase.Cell; -import org.apache.hbase.CellComparator; import org.apache.hbase.codec.prefixtree.decode.DecoderFactory; import org.apache.hbase.codec.prefixtree.encode.PrefixTreeEncoder; import org.apache.hbase.codec.prefixtree.scanner.CellScannerPosition; diff --git a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestRowEncoder.java b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestRowEncoder.java index 8c102f5..b2e81b9 100644 --- a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestRowEncoder.java +++ b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestRowEncoder.java @@ -25,10 +25,10 @@ import java.nio.ByteBuffer; import java.util.Collection; import java.util.List; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; import org.apache.hbase.codec.prefixtree.PrefixTreeBlockMeta; import org.apache.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher; import org.apache.hbase.codec.prefixtree.encode.PrefixTreeEncoder; diff --git a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java index f960527..37afc54 100644 --- a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java +++ b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java @@ -21,10 +21,10 @@ package org.apache.hbase.codec.prefixtree.row.data; import java.util.Collections; import java.util.List; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.CellComparator; import org.apache.hbase.codec.prefixtree.row.BaseTestRowData; import com.google.common.collect.Lists; diff --git a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java index f9ee21b..fd16a7b 100644 --- a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java +++ b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java @@ -20,10 +20,10 @@ package org.apache.hbase.codec.prefixtree.row.data; import java.util.List; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; -import org.apache.hbase.CellComparator; import org.apache.hbase.codec.prefixtree.row.BaseTestRowData; import org.apache.hbase.codec.prefixtree.scanner.CellScannerPosition; import org.apache.hbase.codec.prefixtree.scanner.CellSearcher; diff --git a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSimple.java b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSimple.java index 0be8e8e..64c60ef 100644 --- a/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSimple.java +++ b/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSimple.java @@ -20,11 +20,11 @@ package org.apache.hbase.codec.prefixtree.row.data; import java.util.List; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CollectionUtils; -import org.apache.hbase.Cell; -import org.apache.hbase.CellComparator; import org.apache.hbase.codec.prefixtree.row.BaseTestRowData; import org.apache.hbase.codec.prefixtree.scanner.CellScannerPosition; import org.apache.hbase.codec.prefixtree.scanner.CellSearcher; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java index d4ea6e3..66af074 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java @@ -24,13 +24,13 @@ import java.util.TreeSet; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.util.StringUtils; -import org.apache.hbase.Cell; /** * Emits sorted Puts. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index dce0215..21a9120 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -73,6 +73,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hbase.CompoundConfiguration; import org.apache.hadoop.hbase.exceptions.DroppedSnapshotException; import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -136,7 +137,6 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.io.MultipleIOException; import org.apache.hadoop.util.StringUtils; -import org.apache.hbase.Cell; import org.cliffc.high_scale_lib.Counter; import com.google.common.base.Preconditions; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 93d7054..7461699 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -45,6 +45,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompoundConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -78,7 +79,6 @@ import org.apache.hadoop.hbase.util.CollectionBackedScanner; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.util.StringUtils; -import org.apache.hbase.Cell; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableCollection; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java index bab3267..b256b13 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java @@ -35,6 +35,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -44,7 +45,6 @@ import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.regionserver.MemStoreLAB.Allocation; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hbase.Cell; /** * The MemStore holds in-memory modifications to the Store. Modifications diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java index 9b24ad6..1ff7199 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.client.Delete; @@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProcessorProto import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProcessorProtos.MultiRowMutationProcessorResponse; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; /** * A MultiRowProcessor that performs multiple puts and deletes. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index 0f8484c..6704615 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; @@ -36,7 +37,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; -import org.apache.hbase.Cell; /** * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java index 76fe259..85e4633 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java @@ -41,6 +41,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -64,7 +65,6 @@ import org.apache.hadoop.hbase.rest.model.RowModel; import org.apache.hadoop.hbase.rest.model.ScannerModel; import org.apache.hadoop.hbase.rest.model.TableSchemaModel; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; /** * HTable interface to remote tables accessed via REST gateway diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 8126aa4..ae9a061 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -33,6 +33,7 @@ import com.google.protobuf.Service; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; @@ -71,7 +72,6 @@ import org.apache.hadoop.hbase.security.access.Permission.Action; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.util.Pair; -import org.apache.hbase.Cell; import com.google.common.collect.ImmutableSet; import com.google.common.collect.ListMultimap; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java index f8c3b2e..25e7ed8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java @@ -260,10 +260,10 @@ public class ThriftUtilities { } // Map> - for (Map.Entry> familyEntry: + for (Map.Entry> familyEntry: in.getFamilyMap().entrySet()) { TColumn column = new TColumn(ByteBuffer.wrap(familyEntry.getKey())); - for (org.apache.hbase.Cell cell: familyEntry.getValue()) { + for (org.apache.hadoop.hbase.Cell cell: familyEntry.getValue()) { KeyValue kv = KeyValueUtil.ensureKeyValue(cell); byte[] family = kv.getFamily(); byte[] qualifier = kv.getQualifier(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java index e128e9e..618d4fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java @@ -33,6 +33,7 @@ import java.util.NavigableSet; import com.google.common.collect.ImmutableList; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Get; @@ -52,7 +53,6 @@ import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; -import org.apache.hbase.Cell; /** * A sample region observer that tests the RegionObserver interface. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java index 7decb7e..d667058 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; @@ -41,7 +42,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; -import org.apache.hbase.Cell; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java index 37c0a64..9108802 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hbase.Cell; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; 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 515516e..dc21a21 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,6 @@ 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.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 e9ad5f4..0aff893 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 @@ -45,6 +45,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -93,8 +95,6 @@ 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; -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/TestMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java index b841031..dfc4c5a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; import com.google.common.base.Joiner; import com.google.common.collect.Iterables; 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 beecd42..0246477 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 @@ -37,6 +37,7 @@ import java.util.TreeSet; import org.apache.commons.lang.ArrayUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.KeyValue; @@ -48,7 +49,6 @@ 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.CellComparator; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java index 10dee20..579d249 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java @@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -44,7 +45,6 @@ import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.hbase.Cell; /** * This class runs performance benchmarks for {@link HLog}. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index 43ae8ee..c503712 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -32,6 +32,7 @@ import java.util.NavigableSet; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -56,7 +57,6 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hbase.Cell; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test;