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:47:26 UTC

svn commit: r1512927 [2/2] - in /hbase/branches/0.95/hbase-common/src: main/java/org/apache/hadoop/hbase/types/ test/java/org/apache/hadoop/hbase/types/

Added: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java (added)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,158 @@
+/**
+ * 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;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+
+/**
+ * Wraps an existing {@code DataType} implementation as a terminated
+ * version of itself. This has the useful side-effect of turning an existing
+ * {@code DataType} which is not {@code skippable} into a
+ * {@code skippable} variant.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class TerminatedWrapper<T> implements DataType<T> {
+
+  protected final DataType<T> wrapped;
+  protected final byte[] term;
+
+  /**
+   * Create a terminated version of the {@code wrapped}.
+   * @throws IllegalArgumentException when {@code term} is null or empty.
+   */
+  public TerminatedWrapper(DataType<T> wrapped, byte[] term) {
+    if (null == term || term.length == 0)
+      throw new IllegalArgumentException("terminator must be non-null and non-empty.");
+    this.wrapped = wrapped;
+    wrapped.getOrder().apply(term);
+    this.term = term;
+  }
+
+  /**
+   * Create a terminated version of the {@code wrapped}.
+   * {@code term} is converted to a {@code byte[]} using
+   * {@link Bytes#toBytes(String)}.
+   * @throws IllegalArgumentException when {@code term} is null or empty.
+   */
+  public TerminatedWrapper(DataType<T> wrapped, String term) {
+    this(wrapped, Bytes.toBytes(term));
+  }
+
+  @Override
+  public boolean isOrderPreserving() { return wrapped.isOrderPreserving(); }
+
+  @Override
+  public Order getOrder() { return wrapped.getOrder(); }
+
+  @Override
+  public boolean isNullable() { return wrapped.isNullable(); }
+
+  @Override
+  public boolean isSkippable() { return true; }
+
+  @Override
+  public int encodedLength(T val) {
+    return wrapped.encodedLength(val) + term.length;
+  }
+
+  @Override
+  public Class<T> encodedClass() { return wrapped.encodedClass(); }
+
+  /**
+   * Return the position at which {@code term} begins within {@code src},
+   * or {@code -1} if {@code term} is not found.
+   */
+  protected int terminatorPosition(PositionedByteRange src) {
+    byte[] a = src.getBytes();
+    final int offset = src.getOffset();
+    int i;
+    SKIP: for (i = src.getPosition(); i < src.getLength(); i++) {
+      if (a[offset + i] != term[0]) continue;
+      int j;
+      for (j = 1; j < term.length && offset + j < src.getLength(); j++) {
+        if (a[offset + i + j] != term[j]) continue SKIP;
+      }
+      if (j == term.length) return i; // success
+    }
+    return -1;
+  }
+
+  /**
+   * Skip {@code src}'s position forward over one encoded value.
+   * @param src the buffer containing the encoded value.
+   * @return number of bytes skipped.
+   * @throws IllegalArgumentException when the terminator sequence is not found.
+   */
+  @Override
+  public int skip(PositionedByteRange src) {
+    if (wrapped.isSkippable()) {
+      return wrapped.skip(src);
+    } else {
+      // find the terminator position
+      final int start = src.getPosition();
+      int skipped = terminatorPosition(src);
+      if (-1 == skipped) throw new IllegalArgumentException("Terminator sequence not found.");
+      skipped += term.length;
+      src.setPosition(skipped);
+      return skipped - start;
+    }
+  }
+
+  @Override
+  public T decode(PositionedByteRange src) {
+    if (wrapped.isSkippable()) {
+      return wrapped.decode(src);
+    } else {
+      // find the terminator position
+      int term = terminatorPosition(src);
+      if (-1 == term) throw new IllegalArgumentException("Terminator sequence not found.");
+      byte[] b = new byte[term - src.getPosition()];
+      src.get(b);
+      // TODO: should we assert that b.position == b.length?
+      T ret = wrapped.decode(new SimplePositionedByteRange(b));
+      src.get(this.term);
+      return ret;
+    }
+  }
+
+  /**
+   * Write instance {@code val} into buffer {@code dst}.
+   * @throws IllegalArgumentException when the encoded representation of
+   *           {@code val} contains the {@code term} sequence.
+   */
+  @Override
+  public int encode(PositionedByteRange dst, T val) {
+    final int start = dst.getPosition();
+    int written = wrapped.encode(dst, val);
+    PositionedByteRange b = dst.shallowCopy();
+    b.setLength(dst.getPosition());
+    b.setPosition(start);
+    if (-1 != terminatorPosition(b)) {
+      dst.setPosition(start);
+      throw new IllegalArgumentException("Encoded value contains terminator sequence.");
+    }
+    dst.put(term);
+    return written + term.length;
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java (added)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,84 @@
+/**
+ * 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;
+
+/**
+ * The {@code Union} family of {@link DataType}s encode one of a fixed
+ * set of {@code Object}s. They provide convenience methods which handle
+ * type casting on your behalf.
+ */
+@SuppressWarnings("unchecked")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class Union2<A, B> implements DataType<Object> {
+
+  protected final DataType<A> typeA;
+  protected final DataType<B> typeB;
+
+  /**
+   * Create an instance of {@code Union2} over the set of specified
+   * types.
+   */
+  public Union2(DataType<A> typeA, DataType<B> typeB) {
+    this.typeA = typeA;
+    this.typeB = typeB;
+  }
+
+  @Override
+  public boolean isOrderPreserving() {
+    return typeA.isOrderPreserving() && typeB.isOrderPreserving();
+  }
+
+  @Override
+  public Order getOrder() { return null; }
+
+  @Override
+  public boolean isNullable() {
+    return typeA.isNullable() && typeB.isNullable();
+  }
+
+  @Override
+  public boolean isSkippable() {
+    return typeA.isSkippable() && typeB.isSkippable();
+  }
+
+  @Override
+  public Class<Object> encodedClass() {
+    throw new UnsupportedOperationException(
+      "Union types do not expose a definitive encoded class.");
+  }
+
+  /**
+   * Read an instance of the first type parameter from buffer {@code src}.
+   */
+  public A decodeA(PositionedByteRange src) {
+    return (A) decode(src);
+  }
+
+  /**
+   * Read an instance of the second type parameter from buffer {@code src}.
+   */
+  public B decodeB(PositionedByteRange src) {
+    return (B) decode(src);
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java (added)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,71 @@
+/**
+ * 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;
+
+/**
+ * The {@code Union} family of {@link DataType}s encode one of a fixed
+ * collection of Objects. They provide convenience methods which handle type
+ * casting on your behalf.
+ * @see Union2
+ */
+@SuppressWarnings("unchecked")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class Union3<A, B, C> extends Union2<A, B> {
+
+  protected final DataType<C> typeC;
+
+  /**
+   * Create an instance of {@code Union3} over the set of specified
+   * types.
+   */
+  public Union3(DataType<A> typeA, DataType<B> typeB, DataType<C> typeC) {
+    super(typeA, typeB);
+    this.typeC = typeC;
+  }
+
+  @Override
+  public boolean isOrderPreserving() {
+    return super.isOrderPreserving() && typeC.isOrderPreserving();
+  }
+
+  @Override
+  public Order getOrder() { return null; }
+
+  @Override
+  public boolean isNullable() {
+    return super.isNullable() && typeC.isNullable();
+  }
+
+  @Override
+  public boolean isSkippable() {
+    return super.isSkippable() && typeC.isSkippable();
+  }
+
+  /**
+   * Read an instance of the third type parameter from buffer {@code src}.
+   */
+  public C decodeC(PositionedByteRange src) {
+    return (C) decode(src);
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java (added)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,70 @@
+/**
+ * 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;
+
+/**
+ * The {@code Union} family of {@link DataType}s encode one of a fixed
+ * collection of Objects. They provide convenience methods which handle type
+ * casting on your behalf.
+ */
+@SuppressWarnings("unchecked")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class Union4<A, B, C, D> extends Union3<A, B, C> {
+
+  protected final DataType<D> typeD;
+
+  /**
+   * Create an instance of {@code Union4} over the set of specified
+   * types.
+   */
+  public Union4(DataType<A> typeA, DataType<B> typeB, DataType<C> typeC, DataType<D> typeD) {
+    super(typeA, typeB, typeC);
+    this.typeD = typeD;
+  }
+
+  @Override
+  public boolean isOrderPreserving() {
+    return super.isOrderPreserving() && typeD.isOrderPreserving();
+  }
+
+  @Override
+  public Order getOrder() { return null; }
+
+  @Override
+  public boolean isNullable() {
+    return super.isNullable() && typeD.isNullable();
+  }
+
+  @Override
+  public boolean isSkippable() {
+    return super.isSkippable() && typeD.isSkippable();
+  }
+
+  /**
+   * Read an instance of the fourth type parameter from buffer {@code src}.
+   */
+  public D decodeD(PositionedByteRange src) {
+    return (D) decode(src);
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/package-info.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/package-info.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/package-info.java (added)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/types/package-info.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,50 @@
+/**
+ * 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.
+ */
+
+/**
+ * <p>
+ * This package provides the definition and implementation of HBase's
+ * extensible data type API. {@link org.apache.hadoop.hbase.types.DataType}
+ * is the entry point. Basic type implementations are provided based on two
+ * different encoding schemes: the {@code Raw*} implementations use the
+ * {@code toXXX} methods in {@link org.apache.hadoop.hbase.util.Bytes} and
+ * the {@code Ordered*} implementations use the encoding scheme defined in
+ * {@link org.apache.hadoop.hbase.util.OrderedBytes}. Complex types are also
+ * supported in the form of {@link org.apache.hadoop.hbase.types.Struct} and
+ * the abstract {@code Union} classes.
+ * </p>
+ * <p>
+ * {@link org.apache.hadoop.hbase.types.DataType} implementations are used to
+ * convert a POJO into a {@code byte[]} while maintaining application-level
+ * constraints over the values produces and consumed. They also provide hints
+ * to consumers about the nature of encoded values as well as the relationship
+ * between different instances. See the class comments on
+ * {@link org.apache.hadoop.hbase.types.DataType} for details.
+ * </p>
+ * <p>
+ * The {@link org.apache.hadoop.hbase.types.DataType} interface is primarily
+ * of use for creating rowkeys and column qualifiers. It can also be used as a
+ * an encoder for primitive values. It does not support concerns of complex
+ * object serialization, concepts like schema version and migration. These
+ * concepts are handled more thoroughly by tools like Thrift, Avro, and
+ * Protobuf.
+ * </p>
+ *
+ * @since 0.95.2
+ */
+package org.apache.hadoop.hbase.types;

Added: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java Sun Aug 11 13:47:24 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestFixedLengthWrapper {
+
+  static final byte[][] VALUES = new byte[][] {
+    Bytes.toBytes(""), Bytes.toBytes("1"), Bytes.toBytes("22"), Bytes.toBytes("333"),
+    Bytes.toBytes("4444"), Bytes.toBytes("55555"), Bytes.toBytes("666666"),
+    Bytes.toBytes("7777777"), Bytes.toBytes("88888888"), Bytes.toBytes("999999999"),
+  };
+
+  /**
+   * all values of {@code limit} are >= max length of a member of
+   * {@code VALUES}.
+   */
+  static final int[] limits = { 9, 12, 15 };
+
+  @Test
+  public void testReadWrite() {
+    for (int limit : limits) {
+      PositionedByteRange buff = new SimplePositionedByteRange(limit);
+      for (Order ord : new Order[] { Order.ASCENDING, Order.DESCENDING }) {
+        for (byte[] val : VALUES) {
+          buff.setPosition(0);
+          DataType<byte[]> type = new FixedLengthWrapper<byte[]>(new RawBytes(ord), limit);
+          assertEquals(limit, type.encode(buff, val));
+          byte[] expected = Arrays.copyOf(val, limit);
+          buff.setPosition(0);
+          byte[] actual = type.decode(buff);
+          assertArrayEquals(expected, actual);
+          buff.setPosition(0);
+          assertEquals(limit, type.skip(buff));
+        }
+      }
+    }
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInsufficientRemainingRead() {
+    PositionedByteRange buff = new SimplePositionedByteRange(0);
+    DataType<byte[]> type = new FixedLengthWrapper<byte[]>(new RawBytes(), 3);
+    type.decode(buff);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInsufficientRemainingWrite() {
+    PositionedByteRange buff = new SimplePositionedByteRange(0);
+    DataType<byte[]> type = new FixedLengthWrapper<byte[]>(new RawBytes(), 3);
+    type.encode(buff, Bytes.toBytes(""));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testOverflowPassthrough() {
+    PositionedByteRange buff = new SimplePositionedByteRange(3);
+    DataType<byte[]> type = new FixedLengthWrapper<byte[]>(new RawBytes(), 0);
+    type.encode(buff, Bytes.toBytes("foo"));
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java Sun Aug 11 13:47:24 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 static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestOrderedBlob {
+
+  static final byte[][] VALUES = new byte[][] {
+    null, Bytes.toBytes(""), Bytes.toBytes("1"), Bytes.toBytes("22"), Bytes.toBytes("333"),
+    Bytes.toBytes("4444"), Bytes.toBytes("55555"), Bytes.toBytes("666666"),
+    Bytes.toBytes("7777777"), Bytes.toBytes("88888888"), Bytes.toBytes("999999999"),
+  };
+
+  @Test
+  public void testEncodedLength() {
+    PositionedByteRange buff = new SimplePositionedByteRange(20);
+    for (DataType<byte[]> type : new OrderedBlob[] { OrderedBlob.ASCENDING, OrderedBlob.DESCENDING }) {
+      for (byte[] val : VALUES) {
+        buff.setPosition(0);
+        type.encode(buff, val);
+        assertEquals(
+          "encodedLength does not match actual, " + Bytes.toStringBinary(val),
+          buff.getPosition(), type.encodedLength(val));
+      }
+    }
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,52 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestOrderedBlobVar {
+
+  static final byte[][] VALUES = new byte[][] {
+    null, Bytes.toBytes(""), Bytes.toBytes("1"), Bytes.toBytes("22"), Bytes.toBytes("333"),
+    Bytes.toBytes("4444"), Bytes.toBytes("55555"), Bytes.toBytes("666666"),
+    Bytes.toBytes("7777777"), Bytes.toBytes("88888888"), Bytes.toBytes("999999999"),
+  };
+
+  @Test
+  public void testEncodedLength() {
+    PositionedByteRange buff = new SimplePositionedByteRange(20);
+    for (DataType<byte[]> type :
+      new OrderedBlobVar[] { OrderedBlobVar.ASCENDING, OrderedBlobVar.DESCENDING }) {
+      for (byte[] val : VALUES) {
+        buff.setPosition(0);
+        type.encode(buff, val);
+        assertEquals(
+          "encodedLength does not match actual, " + Bytes.toStringBinary(val),
+          buff.getPosition(), type.encodedLength(val));
+      }
+    }
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,48 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestOrderedString {
+
+  static final String[] VALUES =
+      new String[] { null, "", "1", "22", "333", "4444", "55555", "666666",
+    "7777777", "88888888", "999999999" };
+
+  @Test
+  public void testEncodedLength() {
+    PositionedByteRange buff = new SimplePositionedByteRange(20);
+    for (DataType<String> type : new OrderedString[] { OrderedString.ASCENDING, OrderedString.DESCENDING }) {
+      for (String val : VALUES) {
+        buff.setPosition(0);
+        type.encode(buff, val);
+        assertEquals(
+          "encodedLength does not match actual, " + val,
+          buff.getPosition(), type.encodedLength(val));
+      }
+    }
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java Sun Aug 11 13:47:24 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestRawString {
+
+  static final String[] VALUES = new String[] {
+    "", "1", "22", "333", "4444", "55555", "666666", "7777777", "88888888", "999999999",
+  };
+
+  @Test
+  public void testReadWrite() {
+    for (Order ord : new Order[] { Order.ASCENDING, Order.DESCENDING }) {
+      RawString type =
+          Order.ASCENDING == ord ? RawString.ASCENDING : RawString.DESCENDING;
+      for (String val : VALUES) {
+        PositionedByteRange buff = new SimplePositionedByteRange(Bytes.toBytes(val).length);
+        assertEquals(buff.getLength(), type.encode(buff, val));
+        byte[] expected = Bytes.toBytes(val);
+        ord.apply(expected);
+        assertArrayEquals(expected, buff.getBytes());
+        buff.setPosition(0);
+        assertEquals(val, type.decode(buff));
+        buff.setPosition(0);
+        assertEquals(buff.getLength(), type.skip(buff));
+        assertEquals(buff.getLength(), buff.getPosition());
+      }
+    }
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,364 @@
+package org.apache.hadoop.hbase.types;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * This class both tests and demonstrates how to construct compound rowkeys
+ * from a POJO. The code under test is {@link Struct}.
+ * {@link SpecializedPojo1Type1} demonstrates how one might create their own
+ * custom data type extension for an application POJO.
+ */
+@RunWith(Parameterized.class)
+@Category(SmallTests.class)
+public class TestStruct {
+
+  private Struct generic;
+  @SuppressWarnings("rawtypes")
+  private DataType specialized;
+  private Object[][] constructorArgs;
+
+  public TestStruct(Struct generic, @SuppressWarnings("rawtypes") DataType specialized,
+      Object[][] constructorArgs) {
+    this.generic = generic;
+    this.specialized = specialized;
+    this.constructorArgs = constructorArgs;
+  }
+
+  @Parameters
+  public static Collection<Object[]> params() {
+    Object[][] pojo1Args = {
+        new Object[] { "foo", 5,   10.001 },
+        new Object[] { "foo", 100, 7.0    },
+        new Object[] { "foo", 100, 10.001 },
+        new Object[] { "bar", 5,   10.001 },
+        new Object[] { "bar", 100, 10.001 },
+        new Object[] { "baz", 5,   10.001 },
+    };
+
+    Object[][] pojo2Args = {
+        new Object[] { new byte[0], "it".getBytes(), "was", "the".getBytes() },
+        new Object[] { "best".getBytes(), new byte[0], "of", "times,".getBytes() },
+        new Object[] { "it".getBytes(), "was".getBytes(), "", "the".getBytes() },
+        new Object[] { "worst".getBytes(), "of".getBytes(), "times,", new byte[0] },
+        new Object[] { new byte[0], new byte[0], "", new byte[0] },
+    };
+
+    Object[][] params = new Object[][] {
+        { SpecializedPojo1Type1.GENERIC, new SpecializedPojo1Type1(), pojo1Args },
+        { SpecializedPojo2Type1.GENERIC, new SpecializedPojo2Type1(), pojo2Args },
+    };
+    return Arrays.asList(params);
+  }
+
+  static final Comparator<byte[]> NULL_SAFE_BYTES_COMPARATOR =
+      new Comparator<byte[]>() {
+        @Override
+        public int compare(byte[] o1, byte[] o2) {
+          if (o1 == o2) return 0;
+          if (null == o1) return -1;
+          if (null == o2) return 1;
+          return Bytes.compareTo(o1, o2);
+        }
+      };
+
+  /**
+   * A simple object to serialize.
+   */
+  private static class Pojo1 implements Comparable<Pojo1> {
+    final String stringFieldAsc;
+    final int intFieldAsc;
+    final double doubleFieldAsc;
+    final transient String str;
+
+    public Pojo1(Object... argv) {
+      stringFieldAsc = (String) argv[0];
+      intFieldAsc = (Integer) argv[1];
+      doubleFieldAsc = (Double) argv[2];
+      str = new StringBuilder()
+            .append("{ ")
+            .append(null == stringFieldAsc ? "" : "\"")
+            .append(stringFieldAsc)
+            .append(null == stringFieldAsc ? "" : "\"").append(", ")
+            .append(intFieldAsc).append(", ")
+            .append(doubleFieldAsc)
+            .append(" }")
+            .toString();
+    }
+
+    @Override
+    public String toString() {
+      return str;
+    }
+
+    @Override
+    public int compareTo(Pojo1 o) {
+      int cmp = stringFieldAsc.compareTo(o.stringFieldAsc);
+      if (cmp != 0) return cmp;
+      cmp = Integer.valueOf(intFieldAsc).compareTo(Integer.valueOf(o.intFieldAsc));
+      if (cmp != 0) return cmp;
+      return Double.compare(doubleFieldAsc, o.doubleFieldAsc);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (null == o) return false;
+      if (!(o instanceof Pojo1)) return false;
+      Pojo1 that = (Pojo1) o;
+      return 0 == this.compareTo(that);
+    }
+  }
+
+  /**
+   * A simple object to serialize.
+   */
+  private static class Pojo2 implements Comparable<Pojo2> {
+    final byte[] byteField1Asc;
+    final byte[] byteField2Dsc;
+    final String stringFieldDsc;
+    final byte[] byteField3Dsc;
+    final transient String str;
+
+    public Pojo2(Object... vals) {
+      byte[] empty = new byte[0];
+      byteField1Asc = vals.length > 0 ? (byte[]) vals[0] : empty;
+      byteField2Dsc = vals.length > 1 ? (byte[]) vals[1] : empty;
+      stringFieldDsc = vals.length > 2 ? (String) vals[2] : "";
+      byteField3Dsc = vals.length > 3 ? (byte[]) vals[3] : empty;
+      str = new StringBuilder()
+            .append("{ ")
+            .append(Bytes.toStringBinary(byteField1Asc)).append(", ")
+            .append(Bytes.toStringBinary(byteField2Dsc)).append(", ")
+            .append(null == stringFieldDsc ? "" : "\"")
+            .append(stringFieldDsc)
+            .append(null == stringFieldDsc ? "" : "\"").append(", ")
+            .append(Bytes.toStringBinary(byteField3Dsc))
+            .append(" }")
+            .toString();
+    }
+
+    @Override
+    public String toString() {
+      return str;
+    }
+
+    @Override
+    public int compareTo(Pojo2 o) {
+      int cmp = NULL_SAFE_BYTES_COMPARATOR.compare(byteField1Asc, o.byteField1Asc);
+      if (cmp != 0) return cmp;
+      cmp = -NULL_SAFE_BYTES_COMPARATOR.compare(byteField2Dsc, o.byteField2Dsc);
+      if (cmp != 0) return cmp;
+      if (stringFieldDsc == o.stringFieldDsc) cmp = 0;
+      else if (null == stringFieldDsc) cmp = 1;
+      else if (null == o.stringFieldDsc) cmp = -1;
+      else cmp = -stringFieldDsc.compareTo(o.stringFieldDsc);
+      if (cmp != 0) return cmp;
+      return -NULL_SAFE_BYTES_COMPARATOR.compare(byteField3Dsc, o.byteField3Dsc);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (null == o) return false;
+      if (!(o instanceof Pojo2)) return false;
+      Pojo2 that = (Pojo2) o;
+      return 0 == this.compareTo(that);
+    }
+  }
+
+  /**
+   * A custom data type implementation specialized for {@link Pojo1}.
+   */
+  private static class SpecializedPojo1Type1 implements DataType<Pojo1> {
+
+    private static final RawStringTerminated stringField = new RawStringTerminated("/");
+    private static final RawInteger intField = new RawInteger();
+    private static final RawDouble doubleField = new RawDouble();
+
+    /**
+     * The {@link Struct} equivalent of this type.
+     */
+    public static Struct GENERIC =
+        new StructBuilder().add(stringField)
+                           .add(intField)
+                           .add(doubleField)
+                           .toStruct();
+
+    @Override
+    public boolean isOrderPreserving() { return true; }
+
+    @Override
+    public Order getOrder() { return null; }
+
+    @Override
+    public boolean isNullable() { return false; }
+
+    @Override
+    public boolean isSkippable() { return true; }
+
+    @Override
+    public int encodedLength(Pojo1 val) {
+      return
+          stringField.encodedLength(val.stringFieldAsc) +
+          intField.encodedLength(val.intFieldAsc) +
+          doubleField.encodedLength(val.doubleFieldAsc);
+    }
+
+    @Override
+    public Class<Pojo1> encodedClass() { return Pojo1.class; }
+
+    @Override
+    public int skip(PositionedByteRange src) {
+      int skipped = stringField.skip(src);
+      skipped += intField.skip(src);
+      skipped += doubleField.skip(src);
+      return skipped;
+    }
+
+    @Override
+    public Pojo1 decode(PositionedByteRange src) {
+      Object[] ret = new Object[3];
+      ret[0] = stringField.decode(src);
+      ret[1] = intField.decode(src);
+      ret[2] = doubleField.decode(src);
+      return new Pojo1(ret);
+    }
+
+    @Override
+    public int encode(PositionedByteRange dst, Pojo1 val) {
+      int written = stringField.encode(dst, val.stringFieldAsc);
+      written += intField.encode(dst, val.intFieldAsc);
+      written += doubleField.encode(dst, val.doubleFieldAsc);
+      return written;
+    }
+  }
+
+  /**
+   * A custom data type implementation specialized for {@link Pojo2}.
+   */
+  private static class SpecializedPojo2Type1 implements DataType<Pojo2> {
+
+    private static RawBytesTerminated byteField1 = new RawBytesTerminated("/");
+    private static RawBytesTerminated byteField2 =
+        new RawBytesTerminated(Order.DESCENDING, "/");
+    private static RawStringTerminated stringField =
+        new RawStringTerminated(Order.DESCENDING, new byte[] { 0x00 });
+    private static RawBytes byteField3 = RawBytes.DESCENDING;
+
+    /**
+     * The {@link Struct} equivalent of this type.
+     */
+    public static Struct GENERIC =
+        new StructBuilder().add(byteField1)
+                           .add(byteField2)
+                           .add(stringField)
+                           .add(byteField3)
+                           .toStruct();
+
+    @Override
+    public boolean isOrderPreserving() { return true; }
+
+    @Override
+    public Order getOrder() { return null; }
+
+    @Override
+    public boolean isNullable() { return false; }
+
+    @Override
+    public boolean isSkippable() { return true; }
+
+    @Override
+    public int encodedLength(Pojo2 val) {
+      return
+          byteField1.encodedLength(val.byteField1Asc) +
+          byteField2.encodedLength(val.byteField2Dsc) +
+          stringField.encodedLength(val.stringFieldDsc) +
+          byteField3.encodedLength(val.byteField3Dsc);
+    }
+
+    @Override
+    public Class<Pojo2> encodedClass() { return Pojo2.class; }
+
+    @Override
+    public int skip(PositionedByteRange src) {
+      int skipped = byteField1.skip(src);
+      skipped += byteField2.skip(src);
+      skipped += stringField.skip(src);
+      skipped += byteField3.skip(src);
+      return skipped;
+    }
+
+    @Override
+    public Pojo2 decode(PositionedByteRange src) {
+      Object[] ret = new Object[4];
+      ret[0] = byteField1.decode(src);
+      ret[1] = byteField2.decode(src);
+      ret[2] = stringField.decode(src);
+      ret[3] = byteField3.decode(src);
+      return new Pojo2(ret);
+    }
+
+    @Override
+    public int encode(PositionedByteRange dst, Pojo2 val) {
+      int written = byteField1.encode(dst, val.byteField1Asc);
+      written += byteField2.encode(dst, val.byteField2Dsc);
+      written += stringField.encode(dst, val.stringFieldDsc);
+      written += byteField3.encode(dst, val.byteField3Dsc);
+      return written;
+    }
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testOrderPreservation() throws Exception {
+    Object[] vals = new Object[constructorArgs.length];
+    PositionedByteRange[] encodedGeneric = new PositionedByteRange[constructorArgs.length];
+    PositionedByteRange[] encodedSpecialized = new PositionedByteRange[constructorArgs.length];
+    Constructor<?> ctor = specialized.encodedClass().getConstructor(Object[].class);
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = ctor.newInstance(new Object[] { constructorArgs[i] });
+      encodedGeneric[i] = new SimplePositionedByteRange(generic.encodedLength(constructorArgs[i]));
+      encodedSpecialized[i] = new SimplePositionedByteRange(specialized.encodedLength(vals[i]));
+    }
+
+    // populate our arrays
+    for (int i = 0; i < vals.length; i++) {
+      generic.encode(encodedGeneric[i], constructorArgs[i]);
+      encodedGeneric[i].setPosition(0);
+      specialized.encode(encodedSpecialized[i], vals[i]);
+      encodedSpecialized[i].setPosition(0);
+      assertArrayEquals(encodedGeneric[i].getBytes(), encodedSpecialized[i].getBytes());
+    }
+
+    Arrays.sort(vals);
+    Arrays.sort(encodedGeneric);
+    Arrays.sort(encodedSpecialized);
+
+    for (int i = 0; i < vals.length; i++) {
+      assertEquals(
+        "Struct encoder does not preserve sort order at position " + i,
+        vals[i],
+        ctor.newInstance(new Object[] { generic.decode(encodedGeneric[i]) }));
+      assertEquals(
+        "Specialized encoder does not preserve sort order at position " + i,
+        vals[i], specialized.decode(encodedSpecialized[i]));
+    }
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,100 @@
+/**
+ * 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestTerminatedWrapper {
+
+  static final byte[][] VALUES = new byte[][] {
+    Bytes.toBytes(""), Bytes.toBytes("1"), Bytes.toBytes("22"), Bytes.toBytes("333"),
+    Bytes.toBytes("4444"), Bytes.toBytes("55555"), Bytes.toBytes("666666"),
+    Bytes.toBytes("7777777"), Bytes.toBytes("88888888"), Bytes.toBytes("999999999"),
+  };
+
+  static final byte[][] TERMINATORS = new byte[][] { new byte[] { -1 }, Bytes.toBytes("foo") };
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testEmptyDelimiter() {
+    new TerminatedWrapper<byte[]>(new RawBytes(), "");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testNullDelimiter() {
+    new RawBytesTerminated((byte[]) null);
+    // new TerminatedWrapper<byte[]>(new RawBytes(), (byte[]) null);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testEncodedValueContainsTerm() {
+    DataType<byte[]> type = new TerminatedWrapper<byte[]>(new RawBytes(), "foo");
+    PositionedByteRange buff = new SimplePositionedByteRange(16);
+    type.encode(buff, Bytes.toBytes("hello foobar!"));
+  }
+
+  @Test
+  public void testReadWrite() {
+    PositionedByteRange buff = new SimplePositionedByteRange(12);
+    for (Order ord : new Order[] { Order.ASCENDING, Order.DESCENDING }) {
+      for (byte[] term : TERMINATORS) {
+        for (byte[] val : VALUES) {
+          buff.setPosition(0);
+          DataType<byte[]> type = new TerminatedWrapper<byte[]>(new RawBytes(ord), term);
+          assertEquals(val.length + term.length, type.encode(buff, val));
+          buff.setPosition(0);
+          assertArrayEquals(val, type.decode(buff));
+          assertEquals(val.length + term.length, buff.getPosition());
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testSkip() {
+    PositionedByteRange buff = new SimplePositionedByteRange(12);
+    for (Order ord : new Order[] { Order.ASCENDING, Order.DESCENDING }) {
+      for (byte[] term : TERMINATORS) {
+        for (byte[] val : VALUES) {
+          buff.setPosition(0);
+          DataType<byte[]> type = new TerminatedWrapper<byte[]>(new RawBytes(ord), term);
+          int expected = type.encode(buff, val);
+          buff.setPosition(0);
+          assertEquals(expected, type.skip(buff));
+          assertEquals(expected, buff.getPosition());
+        }
+      }
+    }
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidSkip() {
+    PositionedByteRange buff = new SimplePositionedByteRange(Bytes.toBytes("foo"));
+    DataType<byte[]> type = new TerminatedWrapper<byte[]>(new RawBytes(), new byte[] { 0x00 });
+    type.skip(buff);
+  }
+}

Added: hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java?rev=1512927&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java Sun Aug 11 13:47:24 2013
@@ -0,0 +1,139 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestUnion2 {
+
+  /**
+   * An example <code>Union</code>
+   */
+  private static class SampleUnion1 extends Union2<Integer, String> {
+
+    private static final byte IS_INTEGER = 0x00;
+    private static final byte IS_STRING  = 0x01;
+
+    public SampleUnion1() {
+      super(new RawInteger(), new RawStringTerminated(Order.DESCENDING, "."));
+    }
+
+    @Override
+    public int skip(PositionedByteRange src) {
+      switch (src.get()) {
+        case IS_INTEGER:
+          return 1 + typeA.skip(src);
+        case IS_STRING:
+          return 1 + typeB.skip(src);
+        default:
+          throw new IllegalArgumentException("Unrecognized encoding format.");
+      }
+    }
+
+    @Override
+    public Object decode(PositionedByteRange src) {
+      switch (src.get()) {
+        case IS_INTEGER:
+          return typeA.decode(src);
+        case IS_STRING:
+          return typeB.decode(src);
+        default:
+          throw new IllegalArgumentException("Unrecognized encoding format.");
+      }
+    }
+
+    @Override
+    public int encodedLength(Object val) {
+      Integer i = null;
+      String s = null;
+      try {
+        i = (Integer) val;
+      } catch (ClassCastException e) {}
+      try {
+        s = (String) val;
+      } catch (ClassCastException e) {}
+
+      if (null != i) return 1 + typeA.encodedLength(i);
+      if (null != s) return 1 + typeB.encodedLength(s);
+      throw new IllegalArgumentException("val is not a valid member of this union.");
+    }
+
+    @Override
+    public int encode(PositionedByteRange dst, Object val) {
+      Integer i = null;
+      String s = null;
+      try {
+        i = (Integer) val;
+      } catch (ClassCastException e) {}
+      try {
+        s = (String) val;
+      } catch (ClassCastException e) {}
+
+      if (null != i) {
+        dst.put(IS_INTEGER);
+        return 1 + typeA.encode(dst, i);
+      } else if (null != s) {
+        dst.put(IS_STRING);
+        return 1 + typeB.encode(dst, s);
+      }
+      else
+        throw new IllegalArgumentException("val is not of a supported type.");
+    }
+  }
+
+  @Test
+  public void testEncodeDecode() {
+    Integer intVal = Integer.valueOf(10);
+    String strVal = "hello";
+    PositionedByteRange buff = new SimplePositionedByteRange(10);
+    SampleUnion1 type = new SampleUnion1();
+
+    type.encode(buff, intVal);
+    buff.setPosition(0);
+    assertTrue(0 == intVal.compareTo(type.decodeA(buff)));
+    buff.setPosition(0);
+    type.encode(buff, strVal);
+    buff.setPosition(0);
+    assertTrue(0 == strVal.compareTo(type.decodeB(buff)));
+  }
+
+  @Test
+  public void testSkip() {
+    Integer intVal = Integer.valueOf(10);
+    String strVal = "hello";
+    PositionedByteRange buff = new SimplePositionedByteRange(10);
+    SampleUnion1 type = new SampleUnion1();
+
+    int len = type.encode(buff, intVal);
+    buff.setPosition(0);
+    assertEquals(len, type.skip(buff));
+    buff.setPosition(0);
+    len = type.encode(buff, strVal);
+    buff.setPosition(0);
+    assertEquals(len, type.skip(buff));
+  }
+}