You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nk...@apache.org on 2013/08/11 15:50:06 UTC

svn commit: r1512929 [1/2] - in /hbase/trunk/hbase-common/src: main/java/org/apache/hadoop/hbase/types/ test/java/org/apache/hadoop/hbase/types/

Author: nkeywal
Date: Sun Aug 11 13:50:05 2013
New Revision: 1512929

URL: http://svn.apache.org/r1512929
Log:
HBASE-8693  DataType: provide extensible type API (Nick Dimiduck)

Added:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/package-info.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/types/
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,122 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * <p>
+ * {@code DataType} is the base class for all HBase data types. Data
+ * type implementations are designed to be serialized to and deserialized from
+ * byte[]. Serialized representations can retain the natural sort ordering of
+ * the source object, when a suitable encoding is supported by the underlying
+ * implementation. This is a desirable feature for use in rowkeys and column
+ * qualifiers.
+ * </p>
+ * <p>
+ * {@code DataType}s are different from Hadoop {@link Writable}s in two
+ * significant ways. First, {@code DataType} describes how to serialize a
+ * value, it does not encapsulate a serialized value. Second, {@code DataType}
+ * implementations provide hints to consumers about relationships between the
+ * POJOs they represent and richness of the encoded representation.
+ * </p>
+ * <p>
+ * Data type instances are designed to be stateless, thread-safe, and reused.
+ * Implementations should provide {@code static final} instances corresponding
+ * to each variation on configurable parameters. This is to encourage and
+ * simplify instance reuse. For instance, order-preserving types should provide
+ * static ASCENDING and DESCENDING instances. It is also encouraged for
+ * implementations operating on Java primitive types to provide primitive
+ * implementations of the {@code encode} and {@code decode} methods. This
+ * advice is a performance consideration to clients reading and writing values
+ * in tight loops.
+ * </p>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface DataType<T> {
+
+  /**
+   * Indicates whether this instance writes encoded {@code byte[]}'s
+   * which preserve the natural sort order of the unencoded value.
+   * @return {@code true} when natural order is preserved,
+   *         {@code false} otherwise.
+   */
+  public boolean isOrderPreserving();
+
+  /**
+   * Retrieve the sort {@link Order} imposed by this data type, or null when
+   * natural ordering is not preserved. Value is either ascending or
+   * descending. Default is assumed to be {@link Order#ASCENDING}.
+   */
+  public Order getOrder();
+
+  /**
+   * Indicates whether this instance supports encoding null values. This
+   * depends on the implementation details of the encoding format. All
+   * {@code DataType}s that support null should treat null as comparing
+   * less than any non-null value for default sort ordering purposes.
+   * @return {@code true} when null is supported, {@code false} otherwise.
+   */
+  public boolean isNullable();
+
+  /**
+   * Indicates whether this instance is able to skip over it's encoded value.
+   * {@code DataType}s that are not skippable can only be used as the
+   * right-most field of a {@link Struct}.
+   */
+  public boolean isSkippable();
+
+  /**
+   * Inform consumers how long the encoded {@code byte[]} will be.
+   * @param val The value to check.
+   * @return the number of bytes required to encode {@code val}.a
+   */
+  public int encodedLength(T val);
+
+  /**
+   * Inform consumers over what type this {@code DataType} operates. Useful
+   * when working with bare {@code DataType} instances.
+   */
+  public Class<T> encodedClass();
+
+  /**
+   * Skip {@code src}'s position forward over one encoded value.
+   * @param src the buffer containing the encoded value.
+   * @return number of bytes skipped.
+   */
+  public int skip(PositionedByteRange src);
+
+  /**
+   * Read an instance of {@code T} from the buffer {@code src}.
+   * @param src the buffer containing the encoded value.
+   */
+  public T decode(PositionedByteRange src);
+
+  /**
+   * Write instance {@code val} into buffer {@code dst}.
+   * @param dst the buffer containing the encoded value.
+   * @param val the value to encode onto {@code dst}.
+   * @return number of bytes written.
+   */
+  public int encode(PositionedByteRange dst, T val);
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,107 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+
+/**
+ * Wraps an existing {@link DataType} implementation as a fixed-length
+ * version of itself. This has the useful side-effect of turning an existing
+ * {@link DataType} which is not {@code skippable} into a {@code skippable}
+ * variant.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class FixedLengthWrapper<T> implements DataType<T> {
+
+  protected final DataType<T> base;
+  protected final int length;
+
+  /**
+   * Create a fixed-length version of the {@code wrapped}.
+   * @param base the {@link DataType} to restrict to a fixed length.
+   * @param length the maximum length (in bytes) for encoded values.
+   */
+  public FixedLengthWrapper(DataType<T> base, int length) {
+    this.base = base;
+    this.length = length;
+  }
+
+  /**
+   * Retrieve the maximum length (in bytes) of encoded values.
+   */
+  public int getLength() { return length; }
+
+  @Override
+  public boolean isOrderPreserving() { return base.isOrderPreserving(); }
+
+  @Override
+  public Order getOrder() { return base.getOrder(); }
+
+  @Override
+  public boolean isNullable() { return base.isNullable(); }
+
+  @Override
+  public boolean isSkippable() { return true; }
+
+  @Override
+  public int encodedLength(T val) { return length; }
+
+  @Override
+  public Class<T> encodedClass() { return base.encodedClass(); }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    src.setPosition(src.getPosition() + this.length);
+    return this.length;
+  }
+
+  @Override
+  public T decode(PositionedByteRange src) {
+    if (src.getRemaining() < length) {
+      throw new IllegalArgumentException("Not enough buffer remaining. src.offset: "
+          + src.getOffset() + " src.length: " + src.getLength() + " src.position: "
+          + src.getPosition() + " max length: " + length);
+    }
+    // create a copy range limited to length bytes. boo.
+    PositionedByteRange b = new SimplePositionedByteRange(length);
+    src.get(b.getBytes());
+    return base.decode(b);
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, T val) {
+    if (dst.getRemaining() < length) {
+      throw new IllegalArgumentException("Not enough buffer remaining. dst.offset: "
+          + dst.getOffset() + " dst.length: " + dst.getLength() + " dst.position: "
+          + dst.getPosition() + " max length: " + length);
+    }
+    int written = base.encode(dst, val);
+    if (written > length) {
+      throw new IllegalArgumentException("Length of encoded value (" + written
+          + ") exceeds max length (" + length + ").");
+    }
+    // TODO: is the zero-padding appropriate?
+    for (; written < length; written++) { dst.put((byte) 0x00); }
+    return written;
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.OrderedBytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * A {@code byte[]} of variable-length. Build on
+ * {@link OrderedBytes#encodeBlobCopy(PositionedByteRange, byte[], int, int, Order)}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OrderedBlob extends OrderedBytesBase<byte[]> {
+
+  public static final OrderedBlob ASCENDING = new OrderedBlob(Order.ASCENDING);
+  public static final OrderedBlob DESCENDING = new OrderedBlob(Order.DESCENDING);
+
+  protected OrderedBlob(Order order) { super(order); }
+
+  @Override
+  public boolean isSkippable() { return false; }
+
+  @Override
+  public int encodedLength(byte[] val) {
+    return null == val ?
+      (Order.ASCENDING == order ? 1 : 2) :
+      (Order.ASCENDING == order ? val.length + 1 : val.length + 2);
+  }
+
+  @Override
+  public Class<byte[]> encodedClass() { return byte[].class; }
+
+  @Override
+  public byte[] decode(PositionedByteRange src) {
+    return OrderedBytes.decodeBlobCopy(src);
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, byte[] val) {
+    return OrderedBytes.encodeBlobCopy(dst, val, order);
+  }
+
+  /**
+   * Write a subset of {@code val} to {@code dst}.
+   */
+  public int encode(PositionedByteRange dst, byte[] val, int voff, int vlen) {
+    return OrderedBytes.encodeBlobCopy(dst, val, voff, vlen, order);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,64 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.OrderedBytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An alternative to {@link OrderedBlob} for use by {@link Struct} fields that
+ * do not terminate the fields list. Built on
+ * {@link OrderedBytes#encodeBlobVar(PositionedByteRange, byte[], int, int, Order)}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class OrderedBlobVar extends OrderedBytesBase<byte[]> {
+
+  public static final OrderedBlobVar ASCENDING = new OrderedBlobVar(Order.ASCENDING);
+  public static final OrderedBlobVar DESCENDING = new OrderedBlobVar(Order.DESCENDING);
+
+  protected OrderedBlobVar(Order order) { super(order); }
+
+  @Override
+  public int encodedLength(byte[] val) {
+    return null == val ? 1 : OrderedBytes.blobVarEncodedLength(val.length);
+  }
+
+  @Override
+  public Class<byte[]> encodedClass() { return byte[].class; }
+
+  @Override
+  public byte[] decode(PositionedByteRange src) {
+    return OrderedBytes.decodeBlobVar(src);
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, byte[] val) {
+    return OrderedBytes.encodeBlobVar(dst, val, order);
+  }
+
+  /**
+   * Write a subset of {@code val} to {@code buff}.
+   */
+  public int encode(PositionedByteRange dst, byte[] val, int voff, int vlen) {
+    return OrderedBytes.encodeBlobVar(dst, val, voff, vlen, order);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,56 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.OrderedBytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * Base class for data types backed by the {@link OrderedBytes} encoding
+ * implementations.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class OrderedBytesBase<T> implements DataType<T> {
+
+  protected final Order order;
+
+  protected OrderedBytesBase(Order order) { this.order = order; }
+
+  @Override
+  public boolean isOrderPreserving() { return true; }
+
+  @Override
+  public Order getOrder() { return order; }
+
+  // almost all OrderedBytes implementations are nullable.
+  @Override
+  public boolean isNullable() { return true; }
+
+  // almost all OrderedBytes implementations are skippable.
+  @Override
+  public boolean isSkippable() { return true; }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    return OrderedBytes.skip(src);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,73 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.OrderedBytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+
+/**
+ * A {@code float} of 32-bits using a fixed-length encoding. Based on
+ * {@link OrderedBytes#encodeFloat32(PositionedByteRange, float, Order)}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OrderedFloat32 extends OrderedBytesBase<Float> {
+
+  public static final OrderedFloat32 ASCENDING = new OrderedFloat32(Order.ASCENDING);
+  public static final OrderedFloat32 DESCENDING = new OrderedFloat32(Order.DESCENDING);
+
+  protected OrderedFloat32(Order order) { super(order); }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public int encodedLength(Float val) { return 5; }
+
+  @Override
+  public Class<Float> encodedClass() { return Float.class; }
+
+  @Override
+  public Float decode(PositionedByteRange src) {
+    return OrderedBytes.decodeFloat32(src);
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, Float val) {
+    if (null == val) throw new IllegalArgumentException("Null values not supported.");
+    return OrderedBytes.encodeFloat32(dst, val, order);
+  }
+
+  /**
+   * Read a {@code float} value from the buffer {@code dst}.
+   */
+  public float decodeFloat(PositionedByteRange dst) {
+    return OrderedBytes.decodeFloat32(dst);
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code buff}.
+   */
+  public int encodeFloat(PositionedByteRange dst, float val) {
+    return OrderedBytes.encodeFloat32(dst, val, order);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,73 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.OrderedBytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+
+/**
+ * A {@code double} of 64-bits using a fixed-length encoding. Built on
+ * {@link OrderedBytes#encodeFloat64(PositionedByteRange, double, Order)}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OrderedFloat64 extends OrderedBytesBase<Double> {
+
+  public static final OrderedFloat64 ASCENDING = new OrderedFloat64(Order.ASCENDING);
+  public static final OrderedFloat64 DESCENDING = new OrderedFloat64(Order.DESCENDING);
+
+  protected OrderedFloat64(Order order) { super(order); }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public int encodedLength(Double val) { return 9; }
+
+  @Override
+  public Class<Double> encodedClass() { return Double.class; }
+
+  @Override
+  public Double decode(PositionedByteRange src) {
+    return OrderedBytes.decodeFloat64(src);
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, Double val) {
+    if (null == val) throw new IllegalArgumentException("Null values not supported.");
+    return OrderedBytes.encodeFloat64(dst, val, order);
+  }
+
+  /**
+   * Read a {@code double} value from the buffer {@code src}.
+   */
+  public double decodeDouble(PositionedByteRange src) {
+    return OrderedBytes.decodeFloat64(src);
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code dst}.
+   */
+  public int encodeDouble(PositionedByteRange dst, double val) {
+    return OrderedBytes.encodeFloat64(dst, val, order);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,73 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.OrderedBytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+
+/**
+ * An {@code int} of 32-bits using a fixed-length encoding. Built on
+ * {@link OrderedBytes#encodeInt32(PositionedByteRange, int, Order)}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OrderedInt32 extends OrderedBytesBase<Integer> {
+
+  public static final OrderedInt32 ASCENDING = new OrderedInt32(Order.ASCENDING);
+  public static final OrderedInt32 DESCENDING = new OrderedInt32(Order.DESCENDING);
+
+  protected OrderedInt32(Order order) { super(order); }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public int encodedLength(Integer val) { return 5; }
+
+  @Override
+  public Class<Integer> encodedClass() { return Integer.class; }
+
+  @Override
+  public Integer decode(PositionedByteRange src) {
+    return OrderedBytes.decodeInt32(src);
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, Integer val) {
+    if (null == val) throw new IllegalArgumentException("Null values not supported.");
+    return OrderedBytes.encodeInt32(dst, val, order);
+  }
+
+  /**
+   * Read an {@code int} value from the buffer {@code src}.
+   */
+  public int decodeInt(PositionedByteRange src) {
+    return OrderedBytes.decodeInt32(src);
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code dst}.
+   */
+  public int encodeInt(PositionedByteRange dst, int val) {
+    return OrderedBytes.encodeInt32(dst, val, order);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,73 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.OrderedBytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+
+/**
+ * A {@code long} of 64-bits using a fixed-length encoding. Built on
+ * {@link OrderedBytes#encodeInt64(PositionedByteRange, long, Order)}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OrderedInt64 extends OrderedBytesBase<Long> {
+
+  public static final OrderedInt64 ASCENDING = new OrderedInt64(Order.ASCENDING);
+  public static final OrderedInt64 DESCENDING = new OrderedInt64(Order.DESCENDING);
+
+  protected OrderedInt64(Order order) { super(order); }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public int encodedLength(Long val) { return 9; }
+
+  @Override
+  public Class<Long> encodedClass() { return Long.class; }
+
+  @Override
+  public Long decode(PositionedByteRange src) {
+    return OrderedBytes.decodeInt64(src);
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, Long val) {
+    if (null == val) throw new IllegalArgumentException("Null values not supported.");
+    return OrderedBytes.encodeInt64(dst, val, order);
+  }
+
+  /**
+   * Read a {@code long} value from the buffer {@code src}.
+   */
+  public long decodeLong(PositionedByteRange src) {
+    return OrderedBytes.decodeInt64(src);
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code dst}.
+   */
+  public int encodeLong(PositionedByteRange dst, long val) {
+    return OrderedBytes.encodeInt64(dst, val, order);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,108 @@
+/**
+ * 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.types;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.OrderedBytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+
+/**
+ * An {@link Number} of arbitrary precision and variable-length encoding. The
+ * resulting length of encoded values is determined by the numerical (base
+ * 100) precision, not absolute value. Use this data type anywhere you would
+ * expect to use a {@code DECIMAL} type, a {@link BigDecimal}, a
+ * {@link BigInteger}, or any time you've parsed floating precision values
+ * from text. Built on {@link OrderedBytes#encodeNumeric(PositionedByteRange, BigDecimal, Order)}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OrderedNumeric extends OrderedBytesBase<Number> {
+
+  public static final OrderedNumeric ASCENDING = new OrderedNumeric(Order.ASCENDING);
+  public static final OrderedNumeric DESCENDING = new OrderedNumeric(Order.DESCENDING);
+
+  protected OrderedNumeric(Order order) { super(order); }
+
+  @Override
+  public int encodedLength(Number val) {
+    // TODO: this could be done better.
+    PositionedByteRange buff = new SimplePositionedByteRange(100);
+    return encode(buff, val);
+  }
+
+  @Override
+  public Class<Number> encodedClass() { return Number.class; }
+
+  @Override
+  public Number decode(PositionedByteRange src) {
+    if (OrderedBytes.isNumericInfinite(src) || OrderedBytes.isNumericNaN(src)) {
+      return OrderedBytes.decodeNumericAsDouble(src);
+    }
+    return OrderedBytes.decodeNumericAsBigDecimal(src);
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, Number val) {
+    if (null == val) {
+      return OrderedBytes.encodeNull(dst, order);
+    } else if (val instanceof BigDecimal) {
+      return OrderedBytes.encodeNumeric(dst, (BigDecimal) val, order);
+    } else if (val instanceof BigInteger) {
+      return OrderedBytes.encodeNumeric(dst, new BigDecimal((BigInteger) val), order);
+    } else if (val instanceof Double || val instanceof Float) {
+      return OrderedBytes.encodeNumeric(dst, val.doubleValue(), order);
+    } else {
+      // TODO: other instances of Numeric to consider?
+      return OrderedBytes.encodeNumeric(dst, val.longValue(), order);
+    }
+  }
+
+  /**
+   * Read a {@code long} value from the buffer {@code src}.
+   */
+  public long decodeLong(PositionedByteRange src) {
+    return OrderedBytes.decodeNumericAsLong(src);
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code dst}.
+   */
+  public int encodeLong(PositionedByteRange dst, long val) {
+    return OrderedBytes.encodeNumeric(dst, val, order);
+  }
+
+  /**
+   * Read a {@code double} value from the buffer {@code src}.
+   */
+  public double decodeDouble(PositionedByteRange src) {
+    return OrderedBytes.decodeNumericAsLong(src);
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code dst}.
+   */
+  public int encodeDouble(PositionedByteRange dst, double val) {
+    return OrderedBytes.encodeNumeric(dst, val, order);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,57 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.OrderedBytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * A {@code String} of variable-length. Built on
+ * {@link OrderedBytes#encodeString(PositionedByteRange, String, Order)}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OrderedString extends OrderedBytesBase<String> {
+
+  public static final OrderedString ASCENDING = new OrderedString(Order.ASCENDING);
+  public static final OrderedString DESCENDING = new OrderedString(Order.DESCENDING);
+
+  protected OrderedString(Order order) { super(order); }
+
+  @Override
+  public int encodedLength(String val) {
+    // TODO: use of UTF8 here is a leaky abstraction.
+    return null == val ? 1 : val.getBytes(OrderedBytes.UTF8).length + 2;
+  }
+
+  @Override
+  public Class<String> encodedClass() { return String.class; }
+
+  @Override
+  public String decode(PositionedByteRange src) {
+    return OrderedBytes.decodeString(src);
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, String val) {
+    return OrderedBytes.encodeString(dst, val, order);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,102 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An {@code DataType} for interacting with variable-length values
+ * encoded using {@link Bytes#putBytes(byte[], int, byte[], int, int)}.
+ * Intended to make it easier to transition away from direct use of
+ * {@link Bytes}.
+ * @see Bytes#putBytes(byte[], int, byte[], int, int)
+ * @see RawBytesTerminated
+ * @see RawBytesFixedLength
+ * @see OrderedBlob
+ * @see OrderedBlobVar
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawBytes implements DataType<byte[]> {
+
+  public static final RawBytes ASCENDING = new RawBytes(Order.ASCENDING);
+  public static final RawBytes DESCENDING = new RawBytes(Order.DESCENDING);
+
+  protected final Order order;
+
+  protected RawBytes() { this.order = Order.ASCENDING; }
+  protected RawBytes(Order order) { this.order = order; }
+
+  @Override
+  public boolean isOrderPreserving() { return true; }
+
+  @Override
+  public Order getOrder() { return order; }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public boolean isSkippable() { return false; }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    int skipped = src.getRemaining();
+    src.setPosition(src.getLength());
+    return skipped;
+  }
+
+  @Override
+  public int encodedLength(byte[] val) { return val.length; }
+
+  @Override
+  public Class<byte[]> encodedClass() { return byte[].class; }
+
+  @Override
+  public byte[] decode(PositionedByteRange src) {
+    return decode(src, src.getRemaining());
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, byte[] val) {
+    return encode(dst, val, 0, val.length);
+  }
+
+  /**
+   * Read a {@code byte[]} from the buffer {@code src}.
+   */
+  public byte[] decode(PositionedByteRange src, int length) {
+    byte[] val = new byte[length];
+    src.get(val);
+    return val;
+  }
+
+  /**
+   * Write {@code val} into {@code dst}, respecting {@code voff} and {@code vlen}.
+   * @return number of bytes written.
+   */
+  public int encode(PositionedByteRange dst, byte[] val, int voff, int vlen) {
+    Bytes.putBytes(dst.getBytes(), dst.getOffset() + dst.getPosition(), val, voff, vlen);
+    dst.setPosition(dst.getPosition() + vlen);
+    return vlen;
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An {@code DataType} that encodes fixed-length values encoded using
+ * {@link Bytes#putBytes(byte[], int, byte[], int, int)}. Intended to make it
+ * easier to transition away from direct use of {@link Bytes}.
+ * @see Bytes#putBytes(byte[], int, byte[], int, int)
+ * @see RawBytes
+ * @see OrderedBlob
+ * @see OrderedBlobVar
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawBytesFixedLength extends FixedLengthWrapper<byte[]> {
+
+  /**
+   * Create a {@code RawBytesFixedLength} using the specified {@code order}
+   * and {@code length}.
+   */
+  public RawBytesFixedLength(Order order, int length) {
+    super(new RawBytes(order), length);
+  }
+
+  /**
+   * Create a {@code RawBytesFixedLength} of the specified {@code length}.
+   */
+  public RawBytesFixedLength(int length) {
+    super(new RawBytes(), length);
+  }
+
+  /**
+   * Read a {@code byte[]} from the buffer {@code src}.
+   */
+  public byte[] decode(PositionedByteRange src, int length) {
+    return ((RawBytes) base).decode(src, length);
+  }
+
+  /**
+   * Write {@code val} into {@code buff}, respecting {@code offset} and
+   * {@code length}.
+   */
+  public int encode(PositionedByteRange dst, byte[] val, int voff, int vlen) {
+    return ((RawBytes) base).encode(dst, val, voff, vlen);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,88 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An {@code DataType} that encodes variable-length values encoded using
+ * {@link Bytes#putBytes(byte[], int, byte[], int, int)}. Includes a
+ * termination marker following the raw {@code byte[]} value. Intended to
+ * make it easier to transition away from direct use of {@link Bytes}.
+ * @see Bytes#putBytes(byte[], int, byte[], int, int)
+ * @see RawBytes
+ * @see OrderedBlob
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawBytesTerminated extends TerminatedWrapper<byte[]> {
+
+  /**
+   * Create a {@code RawBytesTerminated} using the specified terminator and
+   * {@code order}.
+   * @throws IllegalArgumentException if {@code term} is {@code null} or empty.
+   */
+  public RawBytesTerminated(Order order, byte[] term) {
+    super(new RawBytes(order), term);
+  }
+
+  /**
+   * Create a {@code RawBytesTerminated} using the specified terminator and
+   * {@code order}.
+   * @throws IllegalArgumentException if {@code term} is {@code null} or empty.
+   */
+  public RawBytesTerminated(Order order, String term) {
+    super(new RawBytes(order), term);
+  }
+
+  /**
+   * Create a {@code RawBytesTerminated} using the specified terminator.
+   * @throws IllegalArgumentException if {@code term} is {@code null} or empty.
+   */
+  public RawBytesTerminated(byte[] term) {
+    super(new RawBytes(), term);
+  }
+
+  /**
+   * Create a {@code RawBytesTerminated} using the specified terminator.
+   * @throws IllegalArgumentException if {@code term} is {@code null} or empty.
+   */
+  public RawBytesTerminated(String term) {
+    super(new RawBytes(), term);
+  }
+
+  /**
+   * Read a {@code byte[]} from the buffer {@code src}.
+   */
+  public byte[] decode(PositionedByteRange src, int length) {
+    return ((RawBytes) wrapped).decode(src, length);
+  }
+
+  /**
+   * Write {@code val} into {@code dst}, respecting {@code offset} and
+   * {@code length}.
+   * @return number of bytes written.
+   */
+  public int encode(PositionedByteRange dst, byte[] val, int voff, int vlen) {
+    return ((RawBytes) wrapped).encode(dst, val, voff, vlen);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,88 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An {@code DataType} for interacting with values encoded using
+ * {@link Bytes#putDouble(byte[], int, double)}. Intended to make it easier to
+ * transition away from direct use of {@link Bytes}.
+ * @see Bytes#putDouble(byte[], int, double)
+ * @see Bytes#toDouble(byte[])
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawDouble implements DataType<Double> {
+
+  @Override
+  public boolean isOrderPreserving() { return false; }
+
+  @Override
+  public Order getOrder() { return null; }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public boolean isSkippable() { return true; }
+
+  @Override
+  public int encodedLength(Double val) { return Bytes.SIZEOF_DOUBLE; }
+
+  @Override
+  public Class<Double> encodedClass() { return Double.class; }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    src.setPosition(src.getPosition() + Bytes.SIZEOF_DOUBLE);
+    return Bytes.SIZEOF_DOUBLE;
+  }
+
+  @Override
+  public Double decode(PositionedByteRange src) {
+    double val = Bytes.toDouble(src.getBytes(), src.getOffset() + src.getPosition());
+    skip(src);
+    return val;
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, Double val) {
+    Bytes.putDouble(dst.getBytes(), dst.getOffset() + dst.getPosition(), val);
+    return skip(dst);
+  }
+
+  /**
+   * Read a {@code double} value from the buffer {@code buff}.
+   */
+  public double decodeDouble(byte[] buff, int offset) {
+    double val = Bytes.toDouble(buff, offset);
+    return val;
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code buff}.
+   */
+  public int encodeDouble(byte[] buff, int offset, double val) {
+    return Bytes.putDouble(buff, offset, val);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,87 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An {@code DataType} for interacting with values encoded using
+ * {@link Bytes#putFloat(byte[], int, float)}. Intended to make it easier to
+ * transition away from direct use of {@link Bytes}.
+ * @see Bytes#putFloat(byte[], int, float)
+ * @see Bytes#toFloat(byte[])
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawFloat implements DataType<Float> {
+
+  @Override
+  public boolean isOrderPreserving() { return false; }
+
+  @Override
+  public Order getOrder() { return null; }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public boolean isSkippable() { return true; }
+
+  @Override
+  public int encodedLength(Float val) { return Bytes.SIZEOF_FLOAT; }
+
+  @Override
+  public Class<Float> encodedClass() { return Float.class; }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    src.setPosition(src.getPosition() + Bytes.SIZEOF_FLOAT);
+    return Bytes.SIZEOF_FLOAT;
+  }
+
+  @Override
+  public Float decode(PositionedByteRange src) {
+    float val = Bytes.toFloat(src.getBytes(), src.getOffset() + src.getPosition());
+    skip(src);
+    return val;
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, Float val) {
+    Bytes.putFloat(dst.getBytes(), dst.getOffset() + dst.getPosition(), val);
+    return skip(dst);
+  }
+
+  /**
+   * Read a {@code float} value from the buffer {@code buff}.
+   */
+  public float decodeFloat(byte[] buff, int offset) {
+    return Bytes.toFloat(buff, offset);
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code buff}.
+   */
+  public int encodeFloat(byte[] buff, int offset, float val) {
+    return Bytes.putFloat(buff, offset, val);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,87 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An {@code DataType} for interacting with values encoded using
+ * {@link Bytes#putInt(byte[], int, int)}. Intended to make it easier to
+ * transition away from direct use of {@link Bytes}.
+ * @see Bytes#putInt(byte[], int, int)
+ * @see Bytes#toInt(byte[])
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawInteger implements DataType<Integer> {
+
+  @Override
+  public boolean isOrderPreserving() { return false; }
+
+  @Override
+  public Order getOrder() { return null; }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public boolean isSkippable() { return true; }
+
+  @Override
+  public int encodedLength(Integer val) { return Bytes.SIZEOF_INT; }
+
+  @Override
+  public Class<Integer> encodedClass() { return Integer.class; }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    src.setPosition(src.getPosition() + Bytes.SIZEOF_INT);
+    return Bytes.SIZEOF_INT;
+  }
+
+  @Override
+  public Integer decode(PositionedByteRange src) {
+    int val = Bytes.toInt(src.getBytes(), src.getOffset() + src.getPosition());
+    skip(src);
+    return val;
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, Integer val) {
+    Bytes.putInt(dst.getBytes(), dst.getOffset() + dst.getPosition(), val);
+    return skip(dst);
+  }
+
+  /**
+   * Read an {@code int} value from the buffer {@code buff}.
+   */
+  public int decodeInt(byte[] buff, int offset) {
+    return Bytes.toInt(buff, offset);
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code buff}.
+   */
+  public int encodeInt(byte[] buff, int offset, int val) {
+    return Bytes.putInt(buff, offset, val);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,87 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An {@code DataType} for interacting with values encoded using
+ * {@link Bytes#putLong(byte[], int, long)}. Intended to make it easier to
+ * transition away from direct use of {@link Bytes}.
+ * @see Bytes#putLong(byte[], int, long)
+ * @see Bytes#toLong(byte[])
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawLong implements DataType<Long> {
+
+  @Override
+  public boolean isOrderPreserving() { return false; }
+
+  @Override
+  public Order getOrder() { return null; }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public boolean isSkippable() { return true; }
+
+  @Override
+  public int encodedLength(Long val) { return Bytes.SIZEOF_LONG; }
+
+  @Override
+  public Class<Long> encodedClass() { return Long.class; }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    src.setPosition(src.getPosition() + Bytes.SIZEOF_LONG);
+    return Bytes.SIZEOF_LONG;
+  }
+
+  @Override
+  public Long decode(PositionedByteRange src) {
+    long val = Bytes.toLong(src.getBytes(), src.getOffset() + src.getPosition());
+    skip(src);
+    return val;
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, Long val) {
+    Bytes.putLong(dst.getBytes(), dst.getOffset() + dst.getPosition(), val);
+    return skip(dst);
+  }
+
+  /**
+   * Read a {@code long} value from the buffer {@code buff}.
+   */
+  public long decodeLong(byte[] buff, int offset) {
+    return Bytes.toLong(buff, offset);
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code buff}.
+   */
+  public int encodeLong(byte[] buff, int offset, long val) {
+    return Bytes.putLong(buff, offset, val);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,94 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An {@code DataType} for interacting with values encoded using
+ * {@link Bytes#toBytes(String)}. Intended to make it easier to transition
+ * away from direct use of {@link Bytes}.
+ * @see Bytes#toBytes(String)
+ * @see Bytes#toString(byte[])
+ * @see RawStringTerminated
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawString implements DataType<String> {
+
+  public static final RawString ASCENDING = new RawString(Order.ASCENDING);
+  public static final RawString DESCENDING = new RawString(Order.DESCENDING);
+
+  protected final Order order;
+
+  protected RawString() { this.order = Order.ASCENDING; }
+  protected RawString(Order order) { this.order = order; }
+
+  @Override
+  public boolean isOrderPreserving() { return true; }
+
+  @Override
+  public Order getOrder() { return order; }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public boolean isSkippable() { return false; }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    int skipped = src.getRemaining();
+    src.setPosition(src.getLength());
+    return skipped;
+  }
+
+  @Override
+  public int encodedLength(String val) { return Bytes.toBytes(val).length; }
+
+  @Override
+  public Class<String> encodedClass() { return String.class; }
+
+  @Override
+  public String decode(PositionedByteRange src) {
+    if (Order.ASCENDING == this.order) {
+      // avoid unnecessary array copy for ASC case.
+      String val =
+          Bytes.toString(src.getBytes(), src.getOffset() + src.getPosition(), src.getRemaining());
+      src.setPosition(src.getLength());
+      return val;
+    } else {
+      byte[] b = new byte[src.getRemaining()];
+      src.get(b);
+      order.apply(b, 0, b.length);
+      return Bytes.toString(b);
+    }
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, String val) {
+    byte[] s = Bytes.toBytes(val);
+    order.apply(s);
+    dst.put(s);
+    return s.length;
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java Sun Aug 11 13:50:05 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+
+/**
+ * An {@code DataType} that encodes fixed-length values encoded using
+ * {@link Bytes#toBytes(String)}. Intended to make it easier to transition
+ * away from direct use of {@link Bytes}.
+ * @see Bytes#toBytes(String)
+ * @see Bytes#toString(byte[], int, int)
+ * @see RawString
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawStringFixedLength extends FixedLengthWrapper<String> {
+
+  /**
+   * Create a {@code RawStringFixedLength} using the specified
+   * {@code order} and {@code length}.
+   */
+  public RawStringFixedLength(Order order, int length) {
+    super(new RawString(order), length);
+  }
+
+  /**
+   * Create a {@code RawStringFixedLength} of the specified {@code length}.
+   */
+  public RawStringFixedLength(int length) {
+    super(new RawString(), length);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,72 @@
+/**
+ * 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.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+
+/**
+ * An {@code DataType} that encodes variable-length values encoded using
+ * {@link Bytes#toBytes(String)}. Includes a termination marker following the
+ * raw {@code byte[]} value. Intended to make it easier to transition
+ * away from direct use of {@link Bytes}.
+ * @see Bytes#toBytes(String)
+ * @see Bytes#toString(byte[], int, int)
+ * @see RawString
+ * @see OrderedString
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RawStringTerminated extends TerminatedWrapper<String> {
+
+  /**
+   * Create a {@code RawStringTerminated} using the specified terminator and
+   * {@code order}.
+   * @throws IllegalArgumentException if {@code term} is {@code null} or empty.
+   */
+  public RawStringTerminated(Order order, byte[] term) {
+    super(new RawString(order), term);
+  }
+
+  /**
+   * Create a {@code RawStringTerminated} using the specified terminator and
+   * {@code order}.
+   * @throws IllegalArgumentException if {@code term} is {@code null} or empty.
+   */
+  public RawStringTerminated(Order order, String term) {
+    super(new RawString(order), term);
+  }
+
+  /**
+   * Create a {@code RawStringTerminated} using the specified terminator.
+   * @throws IllegalArgumentException if {@code term} is {@code null} or empty.
+   */
+  public RawStringTerminated(byte[] term) {
+    super(new RawString(), term);
+  }
+
+  /**
+   * Create a {@code RawStringTerminated} using the specified terminator.
+   * @throws IllegalArgumentException if {@code term} is {@code null} or empty.
+   */
+  public RawStringTerminated(String term) {
+    super(new RawString(), term);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,165 @@
+/**
+ * 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.types;
+
+import java.util.Iterator;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * <p>
+ * {@code Struct} is a simple {@link DataType} for implementing "compound
+ * rowkey" and "compound qualifier" schema design strategies.
+ * </p>
+ * <h3>Encoding</h3>
+ * <p>
+ * {@code Struct} member values are encoded onto the target byte[] in the order
+ * in which they are declared. A {@code Struct} may be used as a member of
+ * another {@code Struct}. {@code Struct}s are not {@code nullable} but their
+ * component fields may be.
+ * </p>
+ * <h3>Sort Order</h3>
+ * <p>
+ * {@code Struct} instances sort according to the composite order of their
+ * fields, that is, left-to-right and depth-first. This can also be thought of
+ * as lexicographic comparison of concatenated members.
+ * </p>
+ * <p>
+ * {@link StructIterator} is provided as a convenience for consuming the
+ * sequence of values. Users may find it more appropriate to provide their own
+ * custom {@link DataType} for encoding application objects rather than using
+ * this {@code Object[]} implementation. Examples are provided in test.
+ * </p>
+ * @see StructIterator
+ * @see DataType#isNullable()
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class Struct implements DataType<Object[]> {
+
+  @SuppressWarnings("rawtypes")
+  protected final DataType[] fields;
+  protected final boolean isOrderPreserving;
+  protected final boolean isSkippable;
+
+  /**
+   * Create a new {@code Struct} instance defined as the sequence of
+   * {@code HDataType}s in {@code memberTypes}.
+   * <p>
+   * A {@code Struct} is {@code orderPreserving} when all of its fields
+   * are {@code orderPreserving}. A {@code Struct} is {@code skippable} when
+   * all of its fields are {@code skippable}.
+   * </p>
+   */
+  @SuppressWarnings("rawtypes")
+  public Struct(DataType[] memberTypes) {
+    this.fields = memberTypes;
+    // a Struct is not orderPreserving when any of its fields are not.
+    boolean preservesOrder = true;
+    // a Struct is not skippable when any of its fields are not.
+    boolean skippable = true;
+    for (int i = 0; i < this.fields.length; i++) {
+      DataType dt = this.fields[i];
+      if (!dt.isOrderPreserving()) preservesOrder = false;
+      if (i < this.fields.length - 2 && !dt.isSkippable()) {
+        throw new IllegalArgumentException("Field in position " + i
+          + " is not skippable. Non-right-most struct fields must be skippable.");
+      }
+      if (!dt.isSkippable()) skippable = false;
+    }
+    this.isOrderPreserving = preservesOrder;
+    this.isSkippable = skippable;
+  }
+
+  @Override
+  public boolean isOrderPreserving() { return isOrderPreserving; }
+
+  @Override
+  public Order getOrder() { return null; }
+
+  @Override
+  public boolean isNullable() { return false; }
+
+  @Override
+  public boolean isSkippable() { return isSkippable; }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public int encodedLength(Object[] val) {
+    assert fields.length == val.length;
+    int sum = 0;
+    for (int i = 0; i < fields.length; i++)
+      sum += fields[i].encodedLength(val[i]);
+    return sum;
+  }
+
+  @Override
+  public Class<Object[]> encodedClass() { return Object[].class; }
+
+  /**
+   * Retrieve an {@link Iterator} over the values encoded in {@code src}.
+   * {@code src}'s position is consumed by consuming this iterator.
+   */
+  public StructIterator iterator(PositionedByteRange src) {
+    return new StructIterator(src, fields);
+  }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    StructIterator it = iterator(src);
+    int skipped = 0;
+    while (it.hasNext())
+      skipped += it.skip();
+    return skipped;
+  }
+
+  @Override
+  public Object[] decode(PositionedByteRange src) {
+    int i = 0;
+    Object[] ret = new Object[fields.length];
+    Iterator<Object> it = iterator(src);
+    while (it.hasNext())
+      ret[i++] = it.next();
+    return ret;
+  }
+
+  /**
+   * Read the field at {@code index}. {@code src}'s position is not affected.
+   */
+  public Object decode(PositionedByteRange src, int index) {
+    assert index >= 0;
+    StructIterator it = iterator(src.shallowCopy());
+    for (; index > 0; index--)
+      it.skip();
+    return it.next();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public int encode(PositionedByteRange dst, Object[] val) {
+    assert fields.length == val.length;
+    int written = 0;
+    for (int i = 0; i < fields.length; i++) {
+      written += fields[i].encode(dst, val[i]);
+    }
+    return written;
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,54 @@
+/**
+ * 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.types;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A helper for building {@link Struct} instances.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class StructBuilder {
+
+  protected final List<DataType<?>> fields = new ArrayList<DataType<?>>();
+
+  /**
+   * Create an empty {@code StructBuilder}.
+   */
+  public StructBuilder() {}
+
+  /**
+   * Append {@code field} to the sequence of accumulated fields.
+   */
+  public StructBuilder add(DataType<?> field) { fields.add(field); return this; }
+
+  /**
+   * Retrieve the {@link Struct} represented by {@code this}.
+   */
+  public Struct toStruct() { return new Struct(fields.toArray(new DataType<?>[fields.size()])); }
+
+  /**
+   * Reset the sequence of accumulated fields.
+   */
+  public StructBuilder reset() { fields.clear(); return this; }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java?rev=1512929&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java Sun Aug 11 13:50:05 2013
@@ -0,0 +1,92 @@
+/**
+ * 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.types;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * An {@link Iterator} over encoded {@code Struct} members.
+ * <p>
+ * This iterates over each serialized {@code Struct} field from the specified
+ * {@code DataTypes<?>[]} definition. It allows you to read the field or skip
+ * over its serialized bytes using {@link #next()} and {@link #skip()},
+ * respectively. This is in contrast to the {@code Struct} method which allow
+ * you to {@link Struct#decode(PositionedByteRange)} or
+ * {@link Struct#skip(PositionedByteRange)} over the entire {@code Struct} at
+ * once.
+ * </p>
+ * <p>
+ * This iterator may also be used to read bytes from any {@code Struct} for
+ * which the specified {@code DataType<?>[]} is a prefix. For example, if the
+ * specified {@code Struct} definition has a {@link RawInteger} and a
+ * {@link RawStringTerminated} field, you may parse the serialized output
+ * of a {@code Struct} whose fields are {@link RawInteger},
+ * {@link RawStringTerminated}, and {@link RawBytes}. The iterator would
+ * return a number followed by a {@code String}. The trailing {@code byte[]}
+ * would be ignored.
+ * </p>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class StructIterator implements Iterator<Object> {
+
+  protected final PositionedByteRange src;
+  protected int idx = 0;
+  @SuppressWarnings("rawtypes")
+  protected final DataType[] types;
+
+  /**
+   * Construct {@code StructIterator} over the values encoded in {@code src}
+   * using the specified {@code types} definition.
+   * @param src The buffer from which to read encoded values.
+   * @param types The sequence of types to use as the schema for this
+   *          {@code Struct}.
+   */
+  public StructIterator(PositionedByteRange src, @SuppressWarnings("rawtypes") DataType[] types) {
+    this.src = src;
+    this.types = types;
+  }
+
+  @Override
+  public boolean hasNext() {
+    return idx < types.length;
+  }
+
+  @Override
+  public void remove() { throw new UnsupportedOperationException(); }
+
+  @Override
+  public Object next() {
+    if (!hasNext()) throw new NoSuchElementException();
+    return types[idx++].decode(src);
+  }
+
+  /**
+   * Bypass the next encoded value.
+   * @return the number of bytes skipped.
+   */
+  public int skip() {
+    if (!hasNext()) throw new NoSuchElementException();
+    return types[idx++].skip(src);
+  }
+}