Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 9D41BD391 for ; Tue, 26 Feb 2013 06:28:46 +0000 (UTC) Received: (qmail 17199 invoked by uid 500); 26 Feb 2013 06:28:46 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 17163 invoked by uid 500); 26 Feb 2013 06:28:46 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 17152 invoked by uid 99); 26 Feb 2013 06:28:46 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 26 Feb 2013 06:28:46 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 26 Feb 2013 06:28:33 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 99AD32388847; Tue, 26 Feb 2013 06:28:10 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1450046 [1/2] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/java/org/apa... Date: Tue, 26 Feb 2013 06:28:07 -0000 To: commits@hbase.apache.org From: stack@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20130226062810.99AD32388847@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: stack Date: Tue Feb 26 06:28:06 2013 New Revision: 1450046 URL: http://svn.apache.org/r1450046 Log: HBASE-7797 Use consistent package name Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScannable.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScanner.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseEncoder.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CodecException.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/CellOutputStream.java hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java Removed: hbase/trunk/hbase-common/src/main/java/org/apache/hbase/ hbase/trunk/hbase-common/src/test/java/org/apache/hbase/ Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/PrefixTreeSeeker.java hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeCell.java hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/CellSearcher.java hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/BaseTestRowData.java hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestRowEncoder.java hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSimple.java hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java (original) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java Tue Feb 26 06:28:06 2013 @@ -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; Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java (original) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java Tue Feb 26 06:28:06 2013 @@ -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; Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Tue Feb 26 06:28:06 2013 @@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFac 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. 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; Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java (original) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java Tue Feb 26 06:28:06 2013 @@ -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. Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java (original) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java Tue Feb 26 06:28:06 2013 @@ -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; Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java (original) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java Tue Feb 26 06:28:06 2013 @@ -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. Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java (original) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java Tue Feb 26 06:28:06 2013 @@ -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; Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Tue Feb 26 06:28:06 2013 @@ -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.Meth 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; Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java Tue Feb 26 06:28:06 2013 @@ -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(); + +} Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java Tue Feb 26 06:28:06 2013 @@ -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); + } + +} Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScannable.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScannable.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScannable.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScannable.java Tue Feb 26 06:28:06 2013 @@ -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 Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScanner.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScanner.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScanner.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellScanner.java Tue Feb 26 06:28:06 2013 @@ -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(); +} Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java Tue Feb 26 06:28:06 2013 @@ -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(); + } + }; + } +} Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java Tue Feb 26 06:28:06 2013 @@ -39,8 +39,6 @@ import org.apache.hadoop.hbase.util.Byte 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; Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java (original) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java Tue Feb 26 06:28:06 2013 @@ -26,7 +26,6 @@ import org.apache.hadoop.classification. 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; Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java (original) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java Tue Feb 26 06:28:06 2013 @@ -26,8 +26,6 @@ import org.apache.hadoop.hbase.util.Byte 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 Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java Tue Feb 26 06:28:06 2013 @@ -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; + } +} Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseEncoder.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseEncoder.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseEncoder.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseEncoder.java Tue Feb 26 06:28:06 2013 @@ -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 Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java Tue Feb 26 06:28:06 2013 @@ -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 Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java Tue Feb 26 06:28:06 2013 @@ -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); +} Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CodecException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CodecException.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CodecException.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CodecException.java Tue Feb 26 06:28:06 2013 @@ -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 Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java Tue Feb 26 06:28:06 2013 @@ -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 Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/CellOutputStream.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/CellOutputStream.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/CellOutputStream.java (added) +++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/CellOutputStream.java Tue Feb 26 06:28:06 2013 @@ -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 Added: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java (added) +++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodec.java Tue Feb 26 06:28:06 2013 @@ -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 Added: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java?rev=1450046&view=auto ============================================================================== --- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java (added) +++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodec.java Tue Feb 26 06:28:06 2013 @@ -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 Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/PrefixTreeSeeker.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/PrefixTreeSeeker.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/PrefixTreeSeeker.java (original) +++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/PrefixTreeSeeker.java Tue Feb 26 06:28:06 2013 @@ -21,11 +21,11 @@ package org.apache.hbase.codec.prefixtre 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; Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java (original) +++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java Tue Feb 26 06:28:06 2013 @@ -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; Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java (original) +++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java Tue Feb 26 06:28:06 2013 @@ -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; Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeCell.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeCell.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeCell.java (original) +++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/decode/PrefixTreeCell.java Tue Feb 26 06:28:06 2013 @@ -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 Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java (original) +++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java Tue Feb 26 06:28:06 2013 @@ -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; Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/CellSearcher.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/CellSearcher.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/CellSearcher.java (original) +++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/CellSearcher.java Tue Feb 26 06:28:06 2013 @@ -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 Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java (original) +++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java Tue Feb 26 06:28:06 2013 @@ -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. Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/BaseTestRowData.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/BaseTestRowData.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/BaseTestRowData.java (original) +++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/BaseTestRowData.java Tue Feb 26 06:28:06 2013 @@ -20,8 +20,8 @@ package org.apache.hbase.codec.prefixtre 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; Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java (original) +++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java Tue Feb 26 06:28:06 2013 @@ -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; Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestRowEncoder.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestRowEncoder.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestRowEncoder.java (original) +++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/TestRowEncoder.java Tue Feb 26 06:28:06 2013 @@ -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; Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java (original) +++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java Tue Feb 26 06:28:06 2013 @@ -21,10 +21,10 @@ package org.apache.hbase.codec.prefixtre 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; Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java?rev=1450046&r1=1450045&r2=1450046&view=diff ============================================================================== --- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java (original) +++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java Tue Feb 26 06:28:06 2013 @@ -20,10 +20,10 @@ package org.apache.hbase.codec.prefixtre 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;