You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2014/09/09 17:18:30 UTC

svn commit: r1623845 [2/3] - in /hive/trunk: hbase-handler/ hbase-handler/if/ hbase-handler/if/test/ hbase-handler/src/java/org/apache/hadoop/hive/hbase/ hbase-handler/src/java/org/apache/hadoop/hive/hbase/struct/ hbase-handler/src/test/org/apache/hado...

Modified: hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java?rev=1623845&r1=1623844&r2=1623845&view=diff
==============================================================================
--- hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java (original)
+++ hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java Tue Sep  9 15:18:29 2014
@@ -22,23 +22,45 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 import junit.framework.TestCase;
 
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.specific.SpecificDatumWriter;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hive.hbase.avro.Address;
+import org.apache.hadoop.hive.hbase.avro.ContactInfo;
+import org.apache.hadoop.hive.hbase.avro.Employee;
+import org.apache.hadoop.hive.hbase.avro.Gender;
+import org.apache.hadoop.hive.hbase.avro.HomePhone;
+import org.apache.hadoop.hive.hbase.avro.OfficePhone;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.lazy.LazyPrimitive;
+import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.io.BooleanWritable;
@@ -53,6 +75,66 @@ import org.apache.thrift.TException;
  */
 public class TestHBaseSerDe extends TestCase {
 
+  static final byte[] TEST_BYTE_ARRAY = Bytes.toBytes("test");
+
+  private static final String RECORD_SCHEMA = "{\n" +
+      "  \"namespace\": \"testing.test.mctesty\",\n" +
+      "  \"name\": \"oneRecord\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"aRecord\",\n" +
+      "      \"type\":{\"type\":\"record\",\n" +
+      "              \"name\":\"recordWithinARecord\",\n" +
+      "              \"fields\": [\n" +
+      "                 {\n" +
+      "                  \"name\":\"int1\",\n" +
+      "                  \"type\":\"int\"\n" +
+      "                },\n" +
+      "                {\n" +
+      "                  \"name\":\"boolean1\",\n" +
+      "                  \"type\":\"boolean\"\n" +
+      "                },\n" +
+      "                {\n" +
+      "                  \"name\":\"long1\",\n" +
+      "                  \"type\":\"long\"\n" +
+      "                }\n" +
+      "      ]}\n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+
+  private static final String RECORD_SCHEMA_EVOLVED = "{\n" +
+      "  \"namespace\": \"testing.test.mctesty\",\n" +
+      "  \"name\": \"oneRecord\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"aRecord\",\n" +
+      "      \"type\":{\"type\":\"record\",\n" +
+      "              \"name\":\"recordWithinARecord\",\n" +
+      "              \"fields\": [\n" +
+      "                 {\n" +
+      "                  \"name\":\"int1\",\n" +
+      "                  \"type\":\"int\"\n" +
+      "                },\n" +
+      "                {\n" +
+      "                  \"name\":\"string1\",\n" +
+      "                  \"type\":\"string\", \"default\": \"test\"\n" +
+      "                },\n" +
+      "                {\n" +
+      "                  \"name\":\"boolean1\",\n" +
+      "                  \"type\":\"boolean\"\n" +
+      "                },\n" +
+      "                {\n" +
+      "                  \"name\":\"long1\",\n" +
+      "                  \"type\":\"long\"\n" +
+      "                }\n" +
+      "      ]}\n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+
   /**
    * Test the default behavior of the Lazy family of objects and object inspectors.
    */
@@ -551,7 +633,7 @@ public class TestHBaseSerDe extends Test
         "key,valint,valbyte,valshort,vallong,valfloat,valdouble,valbool");
     tbl.setProperty(serdeConstants.LIST_COLUMN_TYPES,
         "string:map<int,int>:map<tinyint,tinyint>:map<smallint,smallint>:map<bigint,bigint>:"
-        + "map<float,float>:map<double,double>:map<boolean,boolean>");
+            + "map<float,float>:map<double,double>:map<boolean,boolean>");
     tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING,
         ":key#-,cf-int:#b:b,cf-byte:#b:b,cf-short:#b:b,cf-long:#b:b,cf-float:#b:b,cf-double:#b:b," +
         "cf-bool:#b:b");
@@ -565,7 +647,7 @@ public class TestHBaseSerDe extends Test
         "key,valint,valbyte,valshort,vallong,valfloat,valdouble,valbool");
     tbl.setProperty(serdeConstants.LIST_COLUMN_TYPES,
         "string:map<int,int>:map<tinyint,tinyint>:map<smallint,smallint>:map<bigint,bigint>:"
-        + "map<float,float>:map<double,double>:map<boolean,boolean>");
+            + "map<float,float>:map<double,double>:map<boolean,boolean>");
     tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING,
         ":key#-,cf-int:#-:-,cf-byte:#-:-,cf-short:#-:-,cf-long:#-:-,cf-float:#-:-,cf-double:#-:-," +
         "cf-bool:#-:-");
@@ -636,7 +718,7 @@ public class TestHBaseSerDe extends Test
         "key,valbyte,valshort,valint,vallong,valfloat,valdouble,valstring,valbool");
     tbl.setProperty(serdeConstants.LIST_COLUMN_TYPES,
         "string:map<tinyint,tinyint>:map<smallint,smallint>:map<int,int>:map<bigint,bigint>:"
-        + "map<float,float>:map<double,double>:map<string,string>:map<boolean,boolean>");
+            + "map<float,float>:map<double,double>:map<string,string>:map<boolean,boolean>");
     tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING,
         ":key#s,cf-byte:#-:s,cf-short:#s:-,cf-int:#s:s,cf-long:#-:-,cf-float:#s:-,cf-double:#-:s," +
         "cf-string:#s:s,cf-bool:#-:-");
@@ -650,7 +732,7 @@ public class TestHBaseSerDe extends Test
         "key,valbyte,valshort,valint,vallong,valfloat,valdouble,valstring,valbool");
     tbl.setProperty(serdeConstants.LIST_COLUMN_TYPES,
         "string:map<tinyint,tinyint>:map<smallint,smallint>:map<int,int>:map<bigint,bigint>:"
-        + "map<float,float>:map<double,double>:map<string,string>:map<boolean,boolean>");
+            + "map<float,float>:map<double,double>:map<string,string>:map<boolean,boolean>");
     tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING,
         ":key#s,cf-byte:#s:s,cf-short:#s:s,cf-int:#s:s,cf-long:#s:s,cf-float:#s:s,cf-double:#s:s," +
         "cf-string:#s:s,cf-bool:#s:s");
@@ -934,6 +1016,592 @@ public class TestHBaseSerDe extends Test
     assertEquals("Serialized put:", p.toString(), put.toString());
   }
 
+  public void testHBaseSerDeWithAvroSchemaInline() throws SerDeException, IOException {
+    byte[] cfa = "cola".getBytes();
+
+    byte[] qualAvro = "avro".getBytes();
+
+    byte[] rowKey = Bytes.toBytes("test-row1");
+
+    // Data
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+
+    byte[] avroData = getTestAvroBytesFromSchema(RECORD_SCHEMA);
+
+    kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Result r = new Result(kvs);
+
+    Put p = new Put(rowKey);
+
+    // Post serialization, separators are automatically inserted between different fields in the
+    // struct. Currently there is not way to disable that. So the work around here is to pad the
+    // data with the separator bytes before creating a "Put" object
+    p.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Object[] expectedFieldsData = {new String("test-row1"), new String("[[42, true, 42432234234]]")};
+
+    // Create, initialize, and test the SerDe
+    HBaseSerDe serDe = new HBaseSerDe();
+    Configuration conf = new Configuration();
+    Properties tbl = createPropertiesForHiveAvroSchemaInline();
+    serDe.initialize(conf, tbl);
+
+    deserializeAndSerializeHiveAvro(serDe, r, p, expectedFieldsData);
+  }
+
+  private Properties createPropertiesForHiveAvroSchemaInline() {
+    Properties tbl = new Properties();
+    tbl.setProperty("cola.avro.serialization.type", "avro");
+    tbl.setProperty("cola.avro." + AvroSerdeUtils.SCHEMA_LITERAL, RECORD_SCHEMA);
+    tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING, ":key,cola:avro");
+    tbl.setProperty(HBaseSerDe.HBASE_AUTOGENERATE_STRUCT, "true");
+
+    return tbl;
+  }
+
+  public void testHBaseSerDeWithForwardEvolvedSchema() throws SerDeException, IOException {
+    byte[] cfa = "cola".getBytes();
+
+    byte[] qualAvro = "avro".getBytes();
+
+    byte[] rowKey = Bytes.toBytes("test-row1");
+
+    // Data
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+
+    byte[] avroData = getTestAvroBytesFromSchema(RECORD_SCHEMA);
+
+    kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Result r = new Result(kvs);
+
+    Put p = new Put(rowKey);
+
+    // Post serialization, separators are automatically inserted between different fields in the
+    // struct. Currently there is not way to disable that. So the work around here is to pad the
+    // data with the separator bytes before creating a "Put" object
+    p.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Object[] expectedFieldsData = {new String("test-row1"),
+        new String("[[42, test, true, 42432234234]]")};
+
+    // Create, initialize, and test the SerDe
+    HBaseSerDe serDe = new HBaseSerDe();
+    Configuration conf = new Configuration();
+    Properties tbl = createPropertiesForHiveAvroForwardEvolvedSchema();
+    serDe.initialize(conf, tbl);
+
+    deserializeAndSerializeHiveAvro(serDe, r, p, expectedFieldsData);
+  }
+
+  private Properties createPropertiesForHiveAvroForwardEvolvedSchema() {
+    Properties tbl = new Properties();
+    tbl.setProperty("cola.avro.serialization.type", "avro");
+    tbl.setProperty("cola.avro." + AvroSerdeUtils.SCHEMA_LITERAL, RECORD_SCHEMA_EVOLVED);
+    tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING, ":key,cola:avro");
+    tbl.setProperty(HBaseSerDe.HBASE_AUTOGENERATE_STRUCT, "true");
+
+    return tbl;
+  }
+
+  public void testHBaseSerDeWithBackwardEvolvedSchema() throws SerDeException, IOException {
+    byte[] cfa = "cola".getBytes();
+
+    byte[] qualAvro = "avro".getBytes();
+
+    byte[] rowKey = Bytes.toBytes("test-row1");
+
+    // Data
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+
+    byte[] avroData = getTestAvroBytesFromSchema(RECORD_SCHEMA_EVOLVED);
+
+    kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Result r = new Result(kvs);
+
+    Put p = new Put(rowKey);
+
+    // Post serialization, separators are automatically inserted between different fields in the
+    // struct. Currently there is not way to disable that. So the work around here is to pad the
+    // data with the separator bytes before creating a "Put" object
+    p.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Object[] expectedFieldsData = {new String("test-row1"), new String("[[42, true, 42432234234]]")};
+
+    // Create, initialize, and test the SerDe
+    HBaseSerDe serDe = new HBaseSerDe();
+    Configuration conf = new Configuration();
+    Properties tbl = createPropertiesForHiveAvroBackwardEvolvedSchema();
+    serDe.initialize(conf, tbl);
+
+    deserializeAndSerializeHiveAvro(serDe, r, p, expectedFieldsData);
+  }
+
+  private Properties createPropertiesForHiveAvroBackwardEvolvedSchema() {
+    Properties tbl = new Properties();
+    tbl.setProperty("cola.avro.serialization.type", "avro");
+    tbl.setProperty("cola.avro." + AvroSerdeUtils.SCHEMA_LITERAL, RECORD_SCHEMA);
+    tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING, ":key,cola:avro");
+    tbl.setProperty(HBaseSerDe.HBASE_AUTOGENERATE_STRUCT, "true");
+
+    return tbl;
+  }
+
+  public void testHBaseSerDeWithAvroSerClass() throws SerDeException, IOException {
+    byte[] cfa = "cola".getBytes();
+
+    byte[] qualAvro = "avro".getBytes();
+
+    byte[] rowKey = Bytes.toBytes("test-row1");
+
+    // Data
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+
+    byte[] avroData = getTestAvroBytesFromClass1(1);
+
+    kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Result r = new Result(kvs);
+
+    Put p = new Put(rowKey);
+
+    // Post serialization, separators are automatically inserted between different fields in the
+    // struct. Currently there is not way to disable that. So the work around here is to pad the
+    // data with the separator bytes before creating a "Put" object
+    p.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Object[] expectedFieldsData = {
+        new String("test-row1"),
+        new String(
+            "[Avro Employee1, 11111, 25, FEMALE, [[[Avro First Address1, Avro Second Address1, Avro City1, 123456, 0:[999, 1234567890], null, {testkey=testvalue}], "
+                + "[Avro First Address1, Avro Second Address1, Avro City1, 123456, 0:[999, 1234567890], null, {testkey=testvalue}]], "
+                + "[999, 1234567890], [999, 1234455555]]]")};
+
+    // Create, initialize, and test the SerDe
+    HBaseSerDe serDe = new HBaseSerDe();
+    Configuration conf = new Configuration();
+    Properties tbl = createPropertiesForHiveAvroSerClass();
+    serDe.initialize(conf, tbl);
+
+    deserializeAndSerializeHiveAvro(serDe, r, p, expectedFieldsData);
+  }
+
+  private Properties createPropertiesForHiveAvroSerClass() {
+    Properties tbl = new Properties();
+    tbl.setProperty("cola.avro.serialization.type", "avro");
+    tbl.setProperty("cola.avro." + serdeConstants.SERIALIZATION_CLASS,
+        "org.apache.hadoop.hive.hbase.avro.Employee");
+    tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING, ":key,cola:avro");
+    tbl.setProperty(HBaseSerDe.HBASE_AUTOGENERATE_STRUCT, "true");
+
+    return tbl;
+  }
+
+  public void testHBaseSerDeWithAvroSchemaUrl() throws SerDeException, IOException {
+    byte[] cfa = "cola".getBytes();
+
+    byte[] qualAvro = "avro".getBytes();
+
+    byte[] rowKey = Bytes.toBytes("test-row1");
+
+    // Data
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+
+    byte[] avroData = getTestAvroBytesFromSchema(RECORD_SCHEMA);
+
+    kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Result r = new Result(kvs);
+
+    Put p = new Put(rowKey);
+
+    // Post serialization, separators are automatically inserted between different fields in the
+    // struct. Currently there is not way to disable that. So the work around here is to pad the
+    // data with the separator bytes before creating a "Put" object
+    p.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Object[] expectedFieldsData = {new String("test-row1"), new String("[[42, true, 42432234234]]")};
+
+    MiniDFSCluster miniDfs = null;
+
+    try {
+      // MiniDFSCluster litters files and folders all over the place.
+      miniDfs = new MiniDFSCluster(new Configuration(), 1, true, null);
+
+      miniDfs.getFileSystem().mkdirs(new Path("/path/to/schema"));
+      FSDataOutputStream out = miniDfs.getFileSystem().create(
+          new Path("/path/to/schema/schema.avsc"));
+      out.writeBytes(RECORD_SCHEMA);
+      out.close();
+      String onHDFS = miniDfs.getFileSystem().getUri() + "/path/to/schema/schema.avsc";
+
+      // Create, initialize, and test the SerDe
+      HBaseSerDe serDe = new HBaseSerDe();
+      Configuration conf = new Configuration();
+      Properties tbl = createPropertiesForHiveAvroSchemaUrl(onHDFS);
+      serDe.initialize(conf, tbl);
+
+      deserializeAndSerializeHiveAvro(serDe, r, p, expectedFieldsData);
+    } finally {
+      // Teardown the cluster
+      if (miniDfs != null) {
+        miniDfs.shutdown();
+      }
+    }
+  }
+
+  private Properties createPropertiesForHiveAvroSchemaUrl(String schemaUrl) {
+    Properties tbl = new Properties();
+    tbl.setProperty("cola.avro.serialization.type", "avro");
+    tbl.setProperty("cola.avro." + AvroSerdeUtils.SCHEMA_URL, schemaUrl);
+    tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING, ":key,cola:avro");
+    tbl.setProperty(HBaseSerDe.HBASE_AUTOGENERATE_STRUCT, "true");
+
+    return tbl;
+  }
+
+  public void testHBaseSerDeWithAvroExternalSchema() throws SerDeException, IOException {
+    byte[] cfa = "cola".getBytes();
+
+    byte[] qualAvro = "avro".getBytes();
+
+    byte[] rowKey = Bytes.toBytes("test-row1");
+
+    // Data
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+
+    byte[] avroData = getTestAvroBytesFromClass2(1);
+
+    kvs.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Result r = new Result(kvs);
+
+    Put p = new Put(rowKey);
+
+    // Post serialization, separators are automatically inserted between different fields in the
+    // struct. Currently there is not way to disable that. So the work around here is to pad the
+    // data with the separator bytes before creating a "Put" object
+    p.add(new KeyValue(rowKey, cfa, qualAvro, avroData));
+
+    Object[] expectedFieldsData = {
+        new String("test-row1"),
+        new String(
+            "[Avro Employee1, 11111, 25, FEMALE, [[[Avro First Address1, Avro Second Address1, Avro City1, 123456, 0:[999, 1234567890], null, {testkey=testvalue}], [Avro First Address1, Avro Second Address1, Avro City1, 123456, 0:[999, 1234567890], null, {testkey=testvalue}]], "
+                + "[999, 1234567890], [999, 1234455555]]]")};
+
+    // Create, initialize, and test the SerDe
+    HBaseSerDe serDe = new HBaseSerDe();
+    Configuration conf = new Configuration();
+
+    Properties tbl = createPropertiesForHiveAvroExternalSchema();
+    serDe.initialize(conf, tbl);
+
+    deserializeAndSerializeHiveAvro(serDe, r, p, expectedFieldsData);
+  }
+
+  private Properties createPropertiesForHiveAvroExternalSchema() {
+    Properties tbl = new Properties();
+    tbl.setProperty("cola.avro.serialization.type", "avro");
+    tbl.setProperty(AvroSerdeUtils.SCHEMA_RETRIEVER,
+        "org.apache.hadoop.hive.hbase.HBaseTestAvroSchemaRetriever");
+    tbl.setProperty("cola.avro." + serdeConstants.SERIALIZATION_CLASS,
+        "org.apache.hadoop.hive.hbase.avro.Employee");
+    tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING, ":key,cola:avro");
+    tbl.setProperty(HBaseSerDe.HBASE_AUTOGENERATE_STRUCT, "true");
+
+    return tbl;
+  }
+
+  public void testHBaseSerDeWithHiveMapToHBaseAvroColumnFamily() throws Exception {
+    byte[] cfa = "cola".getBytes();
+
+    byte[] qualAvroA = "prefixA_avro1".getBytes();
+    byte[] qualAvroB = "prefixB_avro2".getBytes();
+    byte[] qualAvroC = "prefixB_avro3".getBytes();
+
+    List<Object> qualifiers = new ArrayList<Object>();
+    qualifiers.add(new Text("prefixA_avro1"));
+    qualifiers.add(new Text("prefixB_avro2"));
+    qualifiers.add(new Text("prefixB_avro3"));
+
+    List<Object> expectedQualifiers = new ArrayList<Object>();
+    expectedQualifiers.add(new Text("prefixB_avro2"));
+    expectedQualifiers.add(new Text("prefixB_avro3"));
+
+    byte[] rowKey = Bytes.toBytes("test-row1");
+
+    // Data
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+
+    byte[] avroDataA = getTestAvroBytesFromSchema(RECORD_SCHEMA);
+    byte[] avroDataB = getTestAvroBytesFromClass1(1);
+    byte[] avroDataC = getTestAvroBytesFromClass1(2);
+
+    kvs.add(new KeyValue(rowKey, cfa, qualAvroA, avroDataA));
+    kvs.add(new KeyValue(rowKey, cfa, qualAvroB, avroDataB));
+    kvs.add(new KeyValue(rowKey, cfa, qualAvroC, avroDataC));
+
+    Result r = new Result(kvs);
+
+    Put p = new Put(rowKey);
+
+    // Post serialization, separators are automatically inserted between different fields in the
+    // struct. Currently there is not way to disable that. So the work around here is to pad the
+    // data with the separator bytes before creating a "Put" object
+    p.add(new KeyValue(rowKey, cfa, qualAvroB, Bytes.padTail(avroDataB, 11)));
+    p.add(new KeyValue(rowKey, cfa, qualAvroC, Bytes.padTail(avroDataC, 11)));
+
+    Object[] expectedFieldsData = {
+        new Text("test-row1"),
+        new String(
+            "[Avro Employee1, 11111, 25, FEMALE, [[[Avro First Address1, Avro Second Address1, Avro City1, 123456, 0:[999, 1234567890], null, {testkey=testvalue}], [Avro First Address1, Avro Second Address1, Avro City1, 123456, 0:[999, 1234567890], null, {testkey=testvalue}]], "
+                + "[999, 1234567890], [999, 1234455555]]]"),
+        new String(
+            "[Avro Employee2, 11111, 25, FEMALE, [[[Avro First Address2, Avro Second Address2, Avro City2, 123456, 0:[999, 1234567890], null, {testkey=testvalue}], [Avro First Address2, Avro Second Address2, Avro City2, 123456, 0:[999, 1234567890], null, {testkey=testvalue}]], "
+                + "[999, 1234567890], [999, 1234455555]]]")};
+
+    int[] expectedMapSize = new int[] {2};
+
+    // Create, initialize, and test the SerDe
+    HBaseSerDe serDe = new HBaseSerDe();
+    Configuration conf = new Configuration();
+    Properties tbl = createPropertiesForHiveAvroColumnFamilyMap();
+    serDe.initialize(conf, tbl);
+
+    Object notPresentKey = new Text("prefixA_avro1");
+
+    deserializeAndSerializeHiveStructColumnFamily(serDe, r, p, expectedFieldsData, expectedMapSize,
+        expectedQualifiers,
+        notPresentKey);
+  }
+
+  private Properties createPropertiesForHiveAvroColumnFamilyMap() {
+    Properties tbl = new Properties();
+    tbl.setProperty("cola.prefixB_.serialization.type", "avro");
+    tbl.setProperty("cola.prefixB_." + serdeConstants.SERIALIZATION_CLASS,
+        "org.apache.hadoop.hive.hbase.avro.Employee");
+    tbl.setProperty(HBaseSerDe.HBASE_COLUMNS_MAPPING, "cola:prefixB_.*");
+    tbl.setProperty(HBaseSerDe.HBASE_AUTOGENERATE_STRUCT, "true");
+    tbl.setProperty(LazySimpleSerDe.SERIALIZATION_EXTEND_NESTING_LEVELS, "true");
+
+    return tbl;
+  }
+
+  private void deserializeAndSerializeHiveAvro(HBaseSerDe serDe, Result r, Put p,
+      Object[] expectedFieldsData)
+      throws SerDeException, IOException {
+    StructObjectInspector soi = (StructObjectInspector) serDe.getObjectInspector();
+
+    List<? extends StructField> fieldRefs = soi.getAllStructFieldRefs();
+
+    Object row = serDe.deserialize(new ResultWritable(r));
+
+    for (int j = 0; j < fieldRefs.size(); j++) {
+      Object fieldData = soi.getStructFieldData(row, fieldRefs.get(j));
+      assertNotNull(fieldData);
+      assertEquals(expectedFieldsData[j], fieldData.toString().trim());
+    }
+
+    // Now serialize
+    Put put = ((PutWritable) serDe.serialize(row, soi)).getPut();
+
+    assertNotNull(put);
+    assertEquals(p.getFamilyCellMap(), put.getFamilyCellMap());
+  }
+
+  private void deserializeAndSerializeHiveStructColumnFamily(HBaseSerDe serDe, Result r, Put p,
+      Object[] expectedFieldsData,
+      int[] expectedMapSize, List<Object> expectedQualifiers, Object notPresentKey)
+      throws SerDeException, IOException {
+    StructObjectInspector soi = (StructObjectInspector) serDe.getObjectInspector();
+
+    List<? extends StructField> fieldRefs = soi.getAllStructFieldRefs();
+
+    Object row = serDe.deserialize(new ResultWritable(r));
+
+    int k = 0;
+
+    for (int i = 0; i < fieldRefs.size(); i++) {
+      Object fieldData = soi.getStructFieldData(row, fieldRefs.get(i));
+      assertNotNull(fieldData);
+
+      if (fieldData instanceof LazyPrimitive<?, ?>) {
+        assertEquals(expectedFieldsData[i], ((LazyPrimitive<?, ?>) fieldData).getWritableObject());
+      } else if (fieldData instanceof LazyHBaseCellMap) {
+
+        for (int j = 0; j < ((LazyHBaseCellMap) fieldData).getMapSize(); j++) {
+          assertEquals(expectedFieldsData[k + 1],
+              ((LazyHBaseCellMap) fieldData).getMapValueElement(expectedQualifiers.get(k))
+                  .toString().trim());
+          k++;
+        }
+
+        assertEquals(expectedMapSize[i - 1], ((LazyHBaseCellMap) fieldData).getMapSize());
+
+        // Make sure that the unwanted key is not present in the map
+        assertNull(((LazyHBaseCellMap) fieldData).getMapValueElement(notPresentKey));
+
+      } else {
+        fail("Error: field data not an instance of LazyPrimitive<?, ?> or LazyHBaseCellMap");
+      }
+    }
+
+    SerDeUtils.getJSONString(row, soi);
+
+    // Now serialize
+    Put put = ((PutWritable) serDe.serialize(row, soi)).getPut();
+
+    assertNotNull(put);
+  }
+
+  private byte[] getTestAvroBytesFromSchema(String schemaToUse) throws IOException {
+    Schema s = Schema.parse(schemaToUse);
+    GenericData.Record record = new GenericData.Record(s);
+    GenericData.Record innerRecord = new GenericData.Record(s.getField("aRecord").schema());
+    innerRecord.put("int1", 42);
+    innerRecord.put("boolean1", true);
+    innerRecord.put("long1", 42432234234l);
+
+    if (schemaToUse.equals(RECORD_SCHEMA_EVOLVED)) {
+      innerRecord.put("string1", "new value");
+    }
+
+    record.put("aRecord", innerRecord);
+
+    DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(s);
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+    DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
+    dataFileWriter.create(s, out);
+    dataFileWriter.append(record);
+    dataFileWriter.close();
+
+    byte[] data = out.toByteArray();
+
+    out.close();
+    return data;
+  }
+
+  private byte[] getTestAvroBytesFromClass1(int i) throws IOException {
+    Employee employee = new Employee();
+
+    employee.setEmployeeName("Avro Employee" + i);
+    employee.setEmployeeID(11111L);
+    employee.setGender(Gender.FEMALE);
+    employee.setAge(25L);
+
+    Address address = new Address();
+
+    address.setAddress1("Avro First Address" + i);
+    address.setAddress2("Avro Second Address" + i);
+    address.setCity("Avro City" + i);
+    address.setZipcode(123456L);
+
+    Map<CharSequence, CharSequence> metadata = new HashMap<CharSequence, CharSequence>();
+
+    metadata.put("testkey", "testvalue");
+
+    address.setMetadata(metadata);
+
+    HomePhone hPhone = new HomePhone();
+
+    hPhone.setAreaCode(999L);
+    hPhone.setNumber(1234567890L);
+
+    OfficePhone oPhone = new OfficePhone();
+
+    oPhone.setAreaCode(999L);
+    oPhone.setNumber(1234455555L);
+
+    ContactInfo contact = new ContactInfo();
+
+    List<Address> addresses = new ArrayList<Address>();
+    address.setCounty(hPhone); // set value for the union type
+    addresses.add(address);
+    addresses.add(address);
+
+    contact.setAddress(addresses);
+
+    contact.setHomePhone(hPhone);
+    contact.setOfficePhone(oPhone);
+
+    employee.setContactInfo(contact);
+
+    DatumWriter<Employee> datumWriter = new SpecificDatumWriter<Employee>(Employee.class);
+    DataFileWriter<Employee> dataFileWriter = new DataFileWriter<Employee>(datumWriter);
+
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+    dataFileWriter.create(employee.getSchema(), out);
+    dataFileWriter.append(employee);
+    dataFileWriter.close();
+
+    return out.toByteArray();
+  }
+
+  private byte[] getTestAvroBytesFromClass2(int i) throws IOException {
+    Employee employee = new Employee();
+
+    employee.setEmployeeName("Avro Employee" + i);
+    employee.setEmployeeID(11111L);
+    employee.setGender(Gender.FEMALE);
+    employee.setAge(25L);
+
+    Address address = new Address();
+
+    address.setAddress1("Avro First Address" + i);
+    address.setAddress2("Avro Second Address" + i);
+    address.setCity("Avro City" + i);
+    address.setZipcode(123456L);
+
+    Map<CharSequence, CharSequence> metadata = new HashMap<CharSequence, CharSequence>();
+
+    metadata.put("testkey", "testvalue");
+
+    address.setMetadata(metadata);
+
+    HomePhone hPhone = new HomePhone();
+
+    hPhone.setAreaCode(999L);
+    hPhone.setNumber(1234567890L);
+
+    OfficePhone oPhone = new OfficePhone();
+
+    oPhone.setAreaCode(999L);
+    oPhone.setNumber(1234455555L);
+
+    ContactInfo contact = new ContactInfo();
+
+    List<Address> addresses = new ArrayList<Address>();
+    address.setCounty(hPhone); // set value for the union type
+    addresses.add(address);
+    addresses.add(address);
+
+    contact.setAddress(addresses);
+
+    contact.setHomePhone(hPhone);
+    contact.setOfficePhone(oPhone);
+
+    employee.setContactInfo(contact);
+
+    DatumWriter<Employee> employeeWriter = new SpecificDatumWriter<Employee>(Employee.class);
+
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+    Encoder encoder = EncoderFactory.get().binaryEncoder(out, null);
+
+    // write out a header for the payload
+    out.write(TEST_BYTE_ARRAY);
+
+    employeeWriter.write(employee, encoder);
+
+    encoder.flush();
+
+    return out.toByteArray();
+  }
+
   class TestStruct {
     String f1;
     String f2;

Added: hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Address.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Address.java?rev=1623845&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Address.java (added)
+++ hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Address.java Tue Sep  9 15:18:29 2014
@@ -0,0 +1,465 @@
+/**
+ * Autogenerated by Avro
+ * 
+ * DO NOT EDIT DIRECTLY
+ */
+package org.apache.hadoop.hive.hbase.avro;  
+@SuppressWarnings("all")
+@org.apache.avro.specific.AvroGenerated
+public class Address extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
+  public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"Address\",\"namespace\":\"org.apache.hadoop.hive.hbase.avro\",\"fields\":[{\"name\":\"address1\",\"type\":\"string\"},{\"name\":\"address2\",\"type\":\"string\"},{\"name\":\"city\",\"type\":\"string\"},{\"name\":\"zipcode\",\"type\":\"long\"},{\"name\":\"county\",\"type\":[{\"type\":\"record\",\"name\":\"HomePhone\",\"fields\":[{\"name\":\"areaCode\",\"type\":\"long\"},{\"name\":\"number\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"OfficePhone\",\"fields\":[{\"name\":\"areaCode\",\"type\":\"long\"},{\"name\":\"number\",\"type\":\"long\"}]},\"string\",\"null\"]},{\"name\":\"aliases\",\"type\":[{\"type\":\"array\",\"items\":\"string\"},\"null\"]},{\"name\":\"metadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":\"string\"}]}]}");
+  public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; }
+  @Deprecated public java.lang.CharSequence address1;
+  @Deprecated public java.lang.CharSequence address2;
+  @Deprecated public java.lang.CharSequence city;
+  @Deprecated public long zipcode;
+  @Deprecated public java.lang.Object county;
+  @Deprecated public java.util.List<java.lang.CharSequence> aliases;
+  @Deprecated public java.util.Map<java.lang.CharSequence,java.lang.CharSequence> metadata;
+
+  /**
+   * Default constructor.  Note that this does not initialize fields
+   * to their default values from the schema.  If that is desired then
+   * one should use <code>newBuilder()</code>. 
+   */
+  public Address() {}
+
+  /**
+   * All-args constructor.
+   */
+  public Address(java.lang.CharSequence address1, java.lang.CharSequence address2, java.lang.CharSequence city, java.lang.Long zipcode, java.lang.Object county, java.util.List<java.lang.CharSequence> aliases, java.util.Map<java.lang.CharSequence,java.lang.CharSequence> metadata) {
+    this.address1 = address1;
+    this.address2 = address2;
+    this.city = city;
+    this.zipcode = zipcode;
+    this.county = county;
+    this.aliases = aliases;
+    this.metadata = metadata;
+  }
+
+  public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+  // Used by DatumWriter.  Applications should not call. 
+  public java.lang.Object get(int field$) {
+    switch (field$) {
+    case 0: return address1;
+    case 1: return address2;
+    case 2: return city;
+    case 3: return zipcode;
+    case 4: return county;
+    case 5: return aliases;
+    case 6: return metadata;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+  // Used by DatumReader.  Applications should not call. 
+  @SuppressWarnings(value="unchecked")
+  public void put(int field$, java.lang.Object value$) {
+    switch (field$) {
+    case 0: address1 = (java.lang.CharSequence)value$; break;
+    case 1: address2 = (java.lang.CharSequence)value$; break;
+    case 2: city = (java.lang.CharSequence)value$; break;
+    case 3: zipcode = (java.lang.Long)value$; break;
+    case 4: county = (java.lang.Object)value$; break;
+    case 5: aliases = (java.util.List<java.lang.CharSequence>)value$; break;
+    case 6: metadata = (java.util.Map<java.lang.CharSequence,java.lang.CharSequence>)value$; break;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+
+  /**
+   * Gets the value of the 'address1' field.
+   */
+  public java.lang.CharSequence getAddress1() {
+    return address1;
+  }
+
+  /**
+   * Sets the value of the 'address1' field.
+   * @param value the value to set.
+   */
+  public void setAddress1(java.lang.CharSequence value) {
+    this.address1 = value;
+  }
+
+  /**
+   * Gets the value of the 'address2' field.
+   */
+  public java.lang.CharSequence getAddress2() {
+    return address2;
+  }
+
+  /**
+   * Sets the value of the 'address2' field.
+   * @param value the value to set.
+   */
+  public void setAddress2(java.lang.CharSequence value) {
+    this.address2 = value;
+  }
+
+  /**
+   * Gets the value of the 'city' field.
+   */
+  public java.lang.CharSequence getCity() {
+    return city;
+  }
+
+  /**
+   * Sets the value of the 'city' field.
+   * @param value the value to set.
+   */
+  public void setCity(java.lang.CharSequence value) {
+    this.city = value;
+  }
+
+  /**
+   * Gets the value of the 'zipcode' field.
+   */
+  public java.lang.Long getZipcode() {
+    return zipcode;
+  }
+
+  /**
+   * Sets the value of the 'zipcode' field.
+   * @param value the value to set.
+   */
+  public void setZipcode(java.lang.Long value) {
+    this.zipcode = value;
+  }
+
+  /**
+   * Gets the value of the 'county' field.
+   */
+  public java.lang.Object getCounty() {
+    return county;
+  }
+
+  /**
+   * Sets the value of the 'county' field.
+   * @param value the value to set.
+   */
+  public void setCounty(java.lang.Object value) {
+    this.county = value;
+  }
+
+  /**
+   * Gets the value of the 'aliases' field.
+   */
+  public java.util.List<java.lang.CharSequence> getAliases() {
+    return aliases;
+  }
+
+  /**
+   * Sets the value of the 'aliases' field.
+   * @param value the value to set.
+   */
+  public void setAliases(java.util.List<java.lang.CharSequence> value) {
+    this.aliases = value;
+  }
+
+  /**
+   * Gets the value of the 'metadata' field.
+   */
+  public java.util.Map<java.lang.CharSequence,java.lang.CharSequence> getMetadata() {
+    return metadata;
+  }
+
+  /**
+   * Sets the value of the 'metadata' field.
+   * @param value the value to set.
+   */
+  public void setMetadata(java.util.Map<java.lang.CharSequence,java.lang.CharSequence> value) {
+    this.metadata = value;
+  }
+
+  /** Creates a new Address RecordBuilder */
+  public static org.apache.hadoop.hive.hbase.avro.Address.Builder newBuilder() {
+    return new org.apache.hadoop.hive.hbase.avro.Address.Builder();
+  }
+  
+  /** Creates a new Address RecordBuilder by copying an existing Builder */
+  public static org.apache.hadoop.hive.hbase.avro.Address.Builder newBuilder(org.apache.hadoop.hive.hbase.avro.Address.Builder other) {
+    return new org.apache.hadoop.hive.hbase.avro.Address.Builder(other);
+  }
+  
+  /** Creates a new Address RecordBuilder by copying an existing Address instance */
+  public static org.apache.hadoop.hive.hbase.avro.Address.Builder newBuilder(org.apache.hadoop.hive.hbase.avro.Address other) {
+    return new org.apache.hadoop.hive.hbase.avro.Address.Builder(other);
+  }
+  
+  /**
+   * RecordBuilder for Address instances.
+   */
+  public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase<Address>
+    implements org.apache.avro.data.RecordBuilder<Address> {
+
+    private java.lang.CharSequence address1;
+    private java.lang.CharSequence address2;
+    private java.lang.CharSequence city;
+    private long zipcode;
+    private java.lang.Object county;
+    private java.util.List<java.lang.CharSequence> aliases;
+    private java.util.Map<java.lang.CharSequence,java.lang.CharSequence> metadata;
+
+    /** Creates a new Builder */
+    private Builder() {
+      super(org.apache.hadoop.hive.hbase.avro.Address.SCHEMA$);
+    }
+    
+    /** Creates a Builder by copying an existing Builder */
+    private Builder(org.apache.hadoop.hive.hbase.avro.Address.Builder other) {
+      super(other);
+      if (isValidValue(fields()[0], other.address1)) {
+        this.address1 = data().deepCopy(fields()[0].schema(), other.address1);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.address2)) {
+        this.address2 = data().deepCopy(fields()[1].schema(), other.address2);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.city)) {
+        this.city = data().deepCopy(fields()[2].schema(), other.city);
+        fieldSetFlags()[2] = true;
+      }
+      if (isValidValue(fields()[3], other.zipcode)) {
+        this.zipcode = data().deepCopy(fields()[3].schema(), other.zipcode);
+        fieldSetFlags()[3] = true;
+      }
+      if (isValidValue(fields()[4], other.county)) {
+        this.county = data().deepCopy(fields()[4].schema(), other.county);
+        fieldSetFlags()[4] = true;
+      }
+      if (isValidValue(fields()[5], other.aliases)) {
+        this.aliases = data().deepCopy(fields()[5].schema(), other.aliases);
+        fieldSetFlags()[5] = true;
+      }
+      if (isValidValue(fields()[6], other.metadata)) {
+        this.metadata = data().deepCopy(fields()[6].schema(), other.metadata);
+        fieldSetFlags()[6] = true;
+      }
+    }
+    
+    /** Creates a Builder by copying an existing Address instance */
+    private Builder(org.apache.hadoop.hive.hbase.avro.Address other) {
+            super(org.apache.hadoop.hive.hbase.avro.Address.SCHEMA$);
+      if (isValidValue(fields()[0], other.address1)) {
+        this.address1 = data().deepCopy(fields()[0].schema(), other.address1);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.address2)) {
+        this.address2 = data().deepCopy(fields()[1].schema(), other.address2);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.city)) {
+        this.city = data().deepCopy(fields()[2].schema(), other.city);
+        fieldSetFlags()[2] = true;
+      }
+      if (isValidValue(fields()[3], other.zipcode)) {
+        this.zipcode = data().deepCopy(fields()[3].schema(), other.zipcode);
+        fieldSetFlags()[3] = true;
+      }
+      if (isValidValue(fields()[4], other.county)) {
+        this.county = data().deepCopy(fields()[4].schema(), other.county);
+        fieldSetFlags()[4] = true;
+      }
+      if (isValidValue(fields()[5], other.aliases)) {
+        this.aliases = data().deepCopy(fields()[5].schema(), other.aliases);
+        fieldSetFlags()[5] = true;
+      }
+      if (isValidValue(fields()[6], other.metadata)) {
+        this.metadata = data().deepCopy(fields()[6].schema(), other.metadata);
+        fieldSetFlags()[6] = true;
+      }
+    }
+
+    /** Gets the value of the 'address1' field */
+    public java.lang.CharSequence getAddress1() {
+      return address1;
+    }
+    
+    /** Sets the value of the 'address1' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder setAddress1(java.lang.CharSequence value) {
+      validate(fields()[0], value);
+      this.address1 = value;
+      fieldSetFlags()[0] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'address1' field has been set */
+    public boolean hasAddress1() {
+      return fieldSetFlags()[0];
+    }
+    
+    /** Clears the value of the 'address1' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder clearAddress1() {
+      address1 = null;
+      fieldSetFlags()[0] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'address2' field */
+    public java.lang.CharSequence getAddress2() {
+      return address2;
+    }
+    
+    /** Sets the value of the 'address2' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder setAddress2(java.lang.CharSequence value) {
+      validate(fields()[1], value);
+      this.address2 = value;
+      fieldSetFlags()[1] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'address2' field has been set */
+    public boolean hasAddress2() {
+      return fieldSetFlags()[1];
+    }
+    
+    /** Clears the value of the 'address2' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder clearAddress2() {
+      address2 = null;
+      fieldSetFlags()[1] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'city' field */
+    public java.lang.CharSequence getCity() {
+      return city;
+    }
+    
+    /** Sets the value of the 'city' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder setCity(java.lang.CharSequence value) {
+      validate(fields()[2], value);
+      this.city = value;
+      fieldSetFlags()[2] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'city' field has been set */
+    public boolean hasCity() {
+      return fieldSetFlags()[2];
+    }
+    
+    /** Clears the value of the 'city' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder clearCity() {
+      city = null;
+      fieldSetFlags()[2] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'zipcode' field */
+    public java.lang.Long getZipcode() {
+      return zipcode;
+    }
+    
+    /** Sets the value of the 'zipcode' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder setZipcode(long value) {
+      validate(fields()[3], value);
+      this.zipcode = value;
+      fieldSetFlags()[3] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'zipcode' field has been set */
+    public boolean hasZipcode() {
+      return fieldSetFlags()[3];
+    }
+    
+    /** Clears the value of the 'zipcode' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder clearZipcode() {
+      fieldSetFlags()[3] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'county' field */
+    public java.lang.Object getCounty() {
+      return county;
+    }
+    
+    /** Sets the value of the 'county' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder setCounty(java.lang.Object value) {
+      validate(fields()[4], value);
+      this.county = value;
+      fieldSetFlags()[4] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'county' field has been set */
+    public boolean hasCounty() {
+      return fieldSetFlags()[4];
+    }
+    
+    /** Clears the value of the 'county' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder clearCounty() {
+      county = null;
+      fieldSetFlags()[4] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'aliases' field */
+    public java.util.List<java.lang.CharSequence> getAliases() {
+      return aliases;
+    }
+    
+    /** Sets the value of the 'aliases' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder setAliases(java.util.List<java.lang.CharSequence> value) {
+      validate(fields()[5], value);
+      this.aliases = value;
+      fieldSetFlags()[5] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'aliases' field has been set */
+    public boolean hasAliases() {
+      return fieldSetFlags()[5];
+    }
+    
+    /** Clears the value of the 'aliases' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder clearAliases() {
+      aliases = null;
+      fieldSetFlags()[5] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'metadata' field */
+    public java.util.Map<java.lang.CharSequence,java.lang.CharSequence> getMetadata() {
+      return metadata;
+    }
+    
+    /** Sets the value of the 'metadata' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder setMetadata(java.util.Map<java.lang.CharSequence,java.lang.CharSequence> value) {
+      validate(fields()[6], value);
+      this.metadata = value;
+      fieldSetFlags()[6] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'metadata' field has been set */
+    public boolean hasMetadata() {
+      return fieldSetFlags()[6];
+    }
+    
+    /** Clears the value of the 'metadata' field */
+    public org.apache.hadoop.hive.hbase.avro.Address.Builder clearMetadata() {
+      metadata = null;
+      fieldSetFlags()[6] = false;
+      return this;
+    }
+
+    @Override
+    public Address build() {
+      try {
+        Address record = new Address();
+        record.address1 = fieldSetFlags()[0] ? this.address1 : (java.lang.CharSequence) defaultValue(fields()[0]);
+        record.address2 = fieldSetFlags()[1] ? this.address2 : (java.lang.CharSequence) defaultValue(fields()[1]);
+        record.city = fieldSetFlags()[2] ? this.city : (java.lang.CharSequence) defaultValue(fields()[2]);
+        record.zipcode = fieldSetFlags()[3] ? this.zipcode : (java.lang.Long) defaultValue(fields()[3]);
+        record.county = fieldSetFlags()[4] ? this.county : (java.lang.Object) defaultValue(fields()[4]);
+        record.aliases = fieldSetFlags()[5] ? this.aliases : (java.util.List<java.lang.CharSequence>) defaultValue(fields()[5]);
+        record.metadata = fieldSetFlags()[6] ? this.metadata : (java.util.Map<java.lang.CharSequence,java.lang.CharSequence>) defaultValue(fields()[6]);
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+  }
+}

Added: hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/ContactInfo.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/ContactInfo.java?rev=1623845&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/ContactInfo.java (added)
+++ hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/ContactInfo.java Tue Sep  9 15:18:29 2014
@@ -0,0 +1,250 @@
+/**
+ * Autogenerated by Avro
+ * 
+ * DO NOT EDIT DIRECTLY
+ */
+package org.apache.hadoop.hive.hbase.avro;  
+@SuppressWarnings("all")
+@org.apache.avro.specific.AvroGenerated
+public class ContactInfo extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
+  public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"ContactInfo\",\"namespace\":\"org.apache.hadoop.hive.hbase.avro\",\"fields\":[{\"name\":\"address\",\"type\":[{\"type\":\"array\",\"items\":{\"type\":\"record\",\"name\":\"Address\",\"fields\":[{\"name\":\"address1\",\"type\":\"string\"},{\"name\":\"address2\",\"type\":\"string\"},{\"name\":\"city\",\"type\":\"string\"},{\"name\":\"zipcode\",\"type\":\"long\"},{\"name\":\"county\",\"type\":[{\"type\":\"record\",\"name\":\"HomePhone\",\"fields\":[{\"name\":\"areaCode\",\"type\":\"long\"},{\"name\":\"number\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"OfficePhone\",\"fields\":[{\"name\":\"areaCode\",\"type\":\"long\"},{\"name\":\"number\",\"type\":\"long\"}]},\"string\",\"null\"]},{\"name\":\"aliases\",\"type\":[{\"type\":\"array\",\"items\":\"string\"},\"null\"]},{\"name\":\"metadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":\"string\"}]}
 ]}},\"null\"]},{\"name\":\"homePhone\",\"type\":\"HomePhone\"},{\"name\":\"officePhone\",\"type\":\"OfficePhone\"}]}");
+  public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; }
+  @Deprecated public java.util.List<org.apache.hadoop.hive.hbase.avro.Address> address;
+  @Deprecated public org.apache.hadoop.hive.hbase.avro.HomePhone homePhone;
+  @Deprecated public org.apache.hadoop.hive.hbase.avro.OfficePhone officePhone;
+
+  /**
+   * Default constructor.  Note that this does not initialize fields
+   * to their default values from the schema.  If that is desired then
+   * one should use <code>newBuilder()</code>. 
+   */
+  public ContactInfo() {}
+
+  /**
+   * All-args constructor.
+   */
+  public ContactInfo(java.util.List<org.apache.hadoop.hive.hbase.avro.Address> address, org.apache.hadoop.hive.hbase.avro.HomePhone homePhone, org.apache.hadoop.hive.hbase.avro.OfficePhone officePhone) {
+    this.address = address;
+    this.homePhone = homePhone;
+    this.officePhone = officePhone;
+  }
+
+  public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+  // Used by DatumWriter.  Applications should not call. 
+  public java.lang.Object get(int field$) {
+    switch (field$) {
+    case 0: return address;
+    case 1: return homePhone;
+    case 2: return officePhone;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+  // Used by DatumReader.  Applications should not call. 
+  @SuppressWarnings(value="unchecked")
+  public void put(int field$, java.lang.Object value$) {
+    switch (field$) {
+    case 0: address = (java.util.List<org.apache.hadoop.hive.hbase.avro.Address>)value$; break;
+    case 1: homePhone = (org.apache.hadoop.hive.hbase.avro.HomePhone)value$; break;
+    case 2: officePhone = (org.apache.hadoop.hive.hbase.avro.OfficePhone)value$; break;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+
+  /**
+   * Gets the value of the 'address' field.
+   */
+  public java.util.List<org.apache.hadoop.hive.hbase.avro.Address> getAddress() {
+    return address;
+  }
+
+  /**
+   * Sets the value of the 'address' field.
+   * @param value the value to set.
+   */
+  public void setAddress(java.util.List<org.apache.hadoop.hive.hbase.avro.Address> value) {
+    this.address = value;
+  }
+
+  /**
+   * Gets the value of the 'homePhone' field.
+   */
+  public org.apache.hadoop.hive.hbase.avro.HomePhone getHomePhone() {
+    return homePhone;
+  }
+
+  /**
+   * Sets the value of the 'homePhone' field.
+   * @param value the value to set.
+   */
+  public void setHomePhone(org.apache.hadoop.hive.hbase.avro.HomePhone value) {
+    this.homePhone = value;
+  }
+
+  /**
+   * Gets the value of the 'officePhone' field.
+   */
+  public org.apache.hadoop.hive.hbase.avro.OfficePhone getOfficePhone() {
+    return officePhone;
+  }
+
+  /**
+   * Sets the value of the 'officePhone' field.
+   * @param value the value to set.
+   */
+  public void setOfficePhone(org.apache.hadoop.hive.hbase.avro.OfficePhone value) {
+    this.officePhone = value;
+  }
+
+  /** Creates a new ContactInfo RecordBuilder */
+  public static org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder newBuilder() {
+    return new org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder();
+  }
+  
+  /** Creates a new ContactInfo RecordBuilder by copying an existing Builder */
+  public static org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder newBuilder(org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder other) {
+    return new org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder(other);
+  }
+  
+  /** Creates a new ContactInfo RecordBuilder by copying an existing ContactInfo instance */
+  public static org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder newBuilder(org.apache.hadoop.hive.hbase.avro.ContactInfo other) {
+    return new org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder(other);
+  }
+  
+  /**
+   * RecordBuilder for ContactInfo instances.
+   */
+  public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase<ContactInfo>
+    implements org.apache.avro.data.RecordBuilder<ContactInfo> {
+
+    private java.util.List<org.apache.hadoop.hive.hbase.avro.Address> address;
+    private org.apache.hadoop.hive.hbase.avro.HomePhone homePhone;
+    private org.apache.hadoop.hive.hbase.avro.OfficePhone officePhone;
+
+    /** Creates a new Builder */
+    private Builder() {
+      super(org.apache.hadoop.hive.hbase.avro.ContactInfo.SCHEMA$);
+    }
+    
+    /** Creates a Builder by copying an existing Builder */
+    private Builder(org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder other) {
+      super(other);
+      if (isValidValue(fields()[0], other.address)) {
+        this.address = data().deepCopy(fields()[0].schema(), other.address);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.homePhone)) {
+        this.homePhone = data().deepCopy(fields()[1].schema(), other.homePhone);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.officePhone)) {
+        this.officePhone = data().deepCopy(fields()[2].schema(), other.officePhone);
+        fieldSetFlags()[2] = true;
+      }
+    }
+    
+    /** Creates a Builder by copying an existing ContactInfo instance */
+    private Builder(org.apache.hadoop.hive.hbase.avro.ContactInfo other) {
+            super(org.apache.hadoop.hive.hbase.avro.ContactInfo.SCHEMA$);
+      if (isValidValue(fields()[0], other.address)) {
+        this.address = data().deepCopy(fields()[0].schema(), other.address);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.homePhone)) {
+        this.homePhone = data().deepCopy(fields()[1].schema(), other.homePhone);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.officePhone)) {
+        this.officePhone = data().deepCopy(fields()[2].schema(), other.officePhone);
+        fieldSetFlags()[2] = true;
+      }
+    }
+
+    /** Gets the value of the 'address' field */
+    public java.util.List<org.apache.hadoop.hive.hbase.avro.Address> getAddress() {
+      return address;
+    }
+    
+    /** Sets the value of the 'address' field */
+    public org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder setAddress(java.util.List<org.apache.hadoop.hive.hbase.avro.Address> value) {
+      validate(fields()[0], value);
+      this.address = value;
+      fieldSetFlags()[0] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'address' field has been set */
+    public boolean hasAddress() {
+      return fieldSetFlags()[0];
+    }
+    
+    /** Clears the value of the 'address' field */
+    public org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder clearAddress() {
+      address = null;
+      fieldSetFlags()[0] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'homePhone' field */
+    public org.apache.hadoop.hive.hbase.avro.HomePhone getHomePhone() {
+      return homePhone;
+    }
+    
+    /** Sets the value of the 'homePhone' field */
+    public org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder setHomePhone(org.apache.hadoop.hive.hbase.avro.HomePhone value) {
+      validate(fields()[1], value);
+      this.homePhone = value;
+      fieldSetFlags()[1] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'homePhone' field has been set */
+    public boolean hasHomePhone() {
+      return fieldSetFlags()[1];
+    }
+    
+    /** Clears the value of the 'homePhone' field */
+    public org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder clearHomePhone() {
+      homePhone = null;
+      fieldSetFlags()[1] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'officePhone' field */
+    public org.apache.hadoop.hive.hbase.avro.OfficePhone getOfficePhone() {
+      return officePhone;
+    }
+    
+    /** Sets the value of the 'officePhone' field */
+    public org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder setOfficePhone(org.apache.hadoop.hive.hbase.avro.OfficePhone value) {
+      validate(fields()[2], value);
+      this.officePhone = value;
+      fieldSetFlags()[2] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'officePhone' field has been set */
+    public boolean hasOfficePhone() {
+      return fieldSetFlags()[2];
+    }
+    
+    /** Clears the value of the 'officePhone' field */
+    public org.apache.hadoop.hive.hbase.avro.ContactInfo.Builder clearOfficePhone() {
+      officePhone = null;
+      fieldSetFlags()[2] = false;
+      return this;
+    }
+
+    @Override
+    public ContactInfo build() {
+      try {
+        ContactInfo record = new ContactInfo();
+        record.address = fieldSetFlags()[0] ? this.address : (java.util.List<org.apache.hadoop.hive.hbase.avro.Address>) defaultValue(fields()[0]);
+        record.homePhone = fieldSetFlags()[1] ? this.homePhone : (org.apache.hadoop.hive.hbase.avro.HomePhone) defaultValue(fields()[1]);
+        record.officePhone = fieldSetFlags()[2] ? this.officePhone : (org.apache.hadoop.hive.hbase.avro.OfficePhone) defaultValue(fields()[2]);
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+  }
+}

Added: hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Employee.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Employee.java?rev=1623845&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Employee.java (added)
+++ hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Employee.java Tue Sep  9 15:18:29 2014
@@ -0,0 +1,356 @@
+/**
+ * Autogenerated by Avro
+ * 
+ * DO NOT EDIT DIRECTLY
+ */
+package org.apache.hadoop.hive.hbase.avro;  
+@SuppressWarnings("all")
+@org.apache.avro.specific.AvroGenerated
+public class Employee extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
+  public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"Employee\",\"namespace\":\"org.apache.hadoop.hive.hbase.avro\",\"fields\":[{\"name\":\"employeeName\",\"type\":\"string\"},{\"name\":\"employeeID\",\"type\":\"long\"},{\"name\":\"age\",\"type\":\"long\"},{\"name\":\"gender\",\"type\":{\"type\":\"enum\",\"name\":\"Gender\",\"symbols\":[\"MALE\",\"FEMALE\"]}},{\"name\":\"contactInfo\",\"type\":{\"type\":\"record\",\"name\":\"ContactInfo\",\"fields\":[{\"name\":\"address\",\"type\":[{\"type\":\"array\",\"items\":{\"type\":\"record\",\"name\":\"Address\",\"fields\":[{\"name\":\"address1\",\"type\":\"string\"},{\"name\":\"address2\",\"type\":\"string\"},{\"name\":\"city\",\"type\":\"string\"},{\"name\":\"zipcode\",\"type\":\"long\"},{\"name\":\"county\",\"type\":[{\"type\":\"record\",\"name\":\"HomePhone\",\"fields\":[{\"name\":\"areaCode\",\"type\":\"long\"},{\"name\":\"number\",\"type\":\"long\"}]},{\"type\"
 :\"record\",\"name\":\"OfficePhone\",\"fields\":[{\"name\":\"areaCode\",\"type\":\"long\"},{\"name\":\"number\",\"type\":\"long\"}]},\"string\",\"null\"]},{\"name\":\"aliases\",\"type\":[{\"type\":\"array\",\"items\":\"string\"},\"null\"]},{\"name\":\"metadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":\"string\"}]}]}},\"null\"]},{\"name\":\"homePhone\",\"type\":\"HomePhone\"},{\"name\":\"officePhone\",\"type\":\"OfficePhone\"}]}}]}");
+  public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; }
+  @Deprecated public java.lang.CharSequence employeeName;
+  @Deprecated public long employeeID;
+  @Deprecated public long age;
+  @Deprecated public org.apache.hadoop.hive.hbase.avro.Gender gender;
+  @Deprecated public org.apache.hadoop.hive.hbase.avro.ContactInfo contactInfo;
+
+  /**
+   * Default constructor.  Note that this does not initialize fields
+   * to their default values from the schema.  If that is desired then
+   * one should use <code>newBuilder()</code>. 
+   */
+  public Employee() {}
+
+  /**
+   * All-args constructor.
+   */
+  public Employee(java.lang.CharSequence employeeName, java.lang.Long employeeID, java.lang.Long age, org.apache.hadoop.hive.hbase.avro.Gender gender, org.apache.hadoop.hive.hbase.avro.ContactInfo contactInfo) {
+    this.employeeName = employeeName;
+    this.employeeID = employeeID;
+    this.age = age;
+    this.gender = gender;
+    this.contactInfo = contactInfo;
+  }
+
+  public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+  // Used by DatumWriter.  Applications should not call. 
+  public java.lang.Object get(int field$) {
+    switch (field$) {
+    case 0: return employeeName;
+    case 1: return employeeID;
+    case 2: return age;
+    case 3: return gender;
+    case 4: return contactInfo;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+  // Used by DatumReader.  Applications should not call. 
+  @SuppressWarnings(value="unchecked")
+  public void put(int field$, java.lang.Object value$) {
+    switch (field$) {
+    case 0: employeeName = (java.lang.CharSequence)value$; break;
+    case 1: employeeID = (java.lang.Long)value$; break;
+    case 2: age = (java.lang.Long)value$; break;
+    case 3: gender = (org.apache.hadoop.hive.hbase.avro.Gender)value$; break;
+    case 4: contactInfo = (org.apache.hadoop.hive.hbase.avro.ContactInfo)value$; break;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+
+  /**
+   * Gets the value of the 'employeeName' field.
+   */
+  public java.lang.CharSequence getEmployeeName() {
+    return employeeName;
+  }
+
+  /**
+   * Sets the value of the 'employeeName' field.
+   * @param value the value to set.
+   */
+  public void setEmployeeName(java.lang.CharSequence value) {
+    this.employeeName = value;
+  }
+
+  /**
+   * Gets the value of the 'employeeID' field.
+   */
+  public java.lang.Long getEmployeeID() {
+    return employeeID;
+  }
+
+  /**
+   * Sets the value of the 'employeeID' field.
+   * @param value the value to set.
+   */
+  public void setEmployeeID(java.lang.Long value) {
+    this.employeeID = value;
+  }
+
+  /**
+   * Gets the value of the 'age' field.
+   */
+  public java.lang.Long getAge() {
+    return age;
+  }
+
+  /**
+   * Sets the value of the 'age' field.
+   * @param value the value to set.
+   */
+  public void setAge(java.lang.Long value) {
+    this.age = value;
+  }
+
+  /**
+   * Gets the value of the 'gender' field.
+   */
+  public org.apache.hadoop.hive.hbase.avro.Gender getGender() {
+    return gender;
+  }
+
+  /**
+   * Sets the value of the 'gender' field.
+   * @param value the value to set.
+   */
+  public void setGender(org.apache.hadoop.hive.hbase.avro.Gender value) {
+    this.gender = value;
+  }
+
+  /**
+   * Gets the value of the 'contactInfo' field.
+   */
+  public org.apache.hadoop.hive.hbase.avro.ContactInfo getContactInfo() {
+    return contactInfo;
+  }
+
+  /**
+   * Sets the value of the 'contactInfo' field.
+   * @param value the value to set.
+   */
+  public void setContactInfo(org.apache.hadoop.hive.hbase.avro.ContactInfo value) {
+    this.contactInfo = value;
+  }
+
+  /** Creates a new Employee RecordBuilder */
+  public static org.apache.hadoop.hive.hbase.avro.Employee.Builder newBuilder() {
+    return new org.apache.hadoop.hive.hbase.avro.Employee.Builder();
+  }
+  
+  /** Creates a new Employee RecordBuilder by copying an existing Builder */
+  public static org.apache.hadoop.hive.hbase.avro.Employee.Builder newBuilder(org.apache.hadoop.hive.hbase.avro.Employee.Builder other) {
+    return new org.apache.hadoop.hive.hbase.avro.Employee.Builder(other);
+  }
+  
+  /** Creates a new Employee RecordBuilder by copying an existing Employee instance */
+  public static org.apache.hadoop.hive.hbase.avro.Employee.Builder newBuilder(org.apache.hadoop.hive.hbase.avro.Employee other) {
+    return new org.apache.hadoop.hive.hbase.avro.Employee.Builder(other);
+  }
+  
+  /**
+   * RecordBuilder for Employee instances.
+   */
+  public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase<Employee>
+    implements org.apache.avro.data.RecordBuilder<Employee> {
+
+    private java.lang.CharSequence employeeName;
+    private long employeeID;
+    private long age;
+    private org.apache.hadoop.hive.hbase.avro.Gender gender;
+    private org.apache.hadoop.hive.hbase.avro.ContactInfo contactInfo;
+
+    /** Creates a new Builder */
+    private Builder() {
+      super(org.apache.hadoop.hive.hbase.avro.Employee.SCHEMA$);
+    }
+    
+    /** Creates a Builder by copying an existing Builder */
+    private Builder(org.apache.hadoop.hive.hbase.avro.Employee.Builder other) {
+      super(other);
+      if (isValidValue(fields()[0], other.employeeName)) {
+        this.employeeName = data().deepCopy(fields()[0].schema(), other.employeeName);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.employeeID)) {
+        this.employeeID = data().deepCopy(fields()[1].schema(), other.employeeID);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.age)) {
+        this.age = data().deepCopy(fields()[2].schema(), other.age);
+        fieldSetFlags()[2] = true;
+      }
+      if (isValidValue(fields()[3], other.gender)) {
+        this.gender = data().deepCopy(fields()[3].schema(), other.gender);
+        fieldSetFlags()[3] = true;
+      }
+      if (isValidValue(fields()[4], other.contactInfo)) {
+        this.contactInfo = data().deepCopy(fields()[4].schema(), other.contactInfo);
+        fieldSetFlags()[4] = true;
+      }
+    }
+    
+    /** Creates a Builder by copying an existing Employee instance */
+    private Builder(org.apache.hadoop.hive.hbase.avro.Employee other) {
+            super(org.apache.hadoop.hive.hbase.avro.Employee.SCHEMA$);
+      if (isValidValue(fields()[0], other.employeeName)) {
+        this.employeeName = data().deepCopy(fields()[0].schema(), other.employeeName);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.employeeID)) {
+        this.employeeID = data().deepCopy(fields()[1].schema(), other.employeeID);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.age)) {
+        this.age = data().deepCopy(fields()[2].schema(), other.age);
+        fieldSetFlags()[2] = true;
+      }
+      if (isValidValue(fields()[3], other.gender)) {
+        this.gender = data().deepCopy(fields()[3].schema(), other.gender);
+        fieldSetFlags()[3] = true;
+      }
+      if (isValidValue(fields()[4], other.contactInfo)) {
+        this.contactInfo = data().deepCopy(fields()[4].schema(), other.contactInfo);
+        fieldSetFlags()[4] = true;
+      }
+    }
+
+    /** Gets the value of the 'employeeName' field */
+    public java.lang.CharSequence getEmployeeName() {
+      return employeeName;
+    }
+    
+    /** Sets the value of the 'employeeName' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder setEmployeeName(java.lang.CharSequence value) {
+      validate(fields()[0], value);
+      this.employeeName = value;
+      fieldSetFlags()[0] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'employeeName' field has been set */
+    public boolean hasEmployeeName() {
+      return fieldSetFlags()[0];
+    }
+    
+    /** Clears the value of the 'employeeName' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder clearEmployeeName() {
+      employeeName = null;
+      fieldSetFlags()[0] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'employeeID' field */
+    public java.lang.Long getEmployeeID() {
+      return employeeID;
+    }
+    
+    /** Sets the value of the 'employeeID' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder setEmployeeID(long value) {
+      validate(fields()[1], value);
+      this.employeeID = value;
+      fieldSetFlags()[1] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'employeeID' field has been set */
+    public boolean hasEmployeeID() {
+      return fieldSetFlags()[1];
+    }
+    
+    /** Clears the value of the 'employeeID' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder clearEmployeeID() {
+      fieldSetFlags()[1] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'age' field */
+    public java.lang.Long getAge() {
+      return age;
+    }
+    
+    /** Sets the value of the 'age' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder setAge(long value) {
+      validate(fields()[2], value);
+      this.age = value;
+      fieldSetFlags()[2] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'age' field has been set */
+    public boolean hasAge() {
+      return fieldSetFlags()[2];
+    }
+    
+    /** Clears the value of the 'age' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder clearAge() {
+      fieldSetFlags()[2] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'gender' field */
+    public org.apache.hadoop.hive.hbase.avro.Gender getGender() {
+      return gender;
+    }
+    
+    /** Sets the value of the 'gender' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder setGender(org.apache.hadoop.hive.hbase.avro.Gender value) {
+      validate(fields()[3], value);
+      this.gender = value;
+      fieldSetFlags()[3] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'gender' field has been set */
+    public boolean hasGender() {
+      return fieldSetFlags()[3];
+    }
+    
+    /** Clears the value of the 'gender' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder clearGender() {
+      gender = null;
+      fieldSetFlags()[3] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'contactInfo' field */
+    public org.apache.hadoop.hive.hbase.avro.ContactInfo getContactInfo() {
+      return contactInfo;
+    }
+    
+    /** Sets the value of the 'contactInfo' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder setContactInfo(org.apache.hadoop.hive.hbase.avro.ContactInfo value) {
+      validate(fields()[4], value);
+      this.contactInfo = value;
+      fieldSetFlags()[4] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'contactInfo' field has been set */
+    public boolean hasContactInfo() {
+      return fieldSetFlags()[4];
+    }
+    
+    /** Clears the value of the 'contactInfo' field */
+    public org.apache.hadoop.hive.hbase.avro.Employee.Builder clearContactInfo() {
+      contactInfo = null;
+      fieldSetFlags()[4] = false;
+      return this;
+    }
+
+    @Override
+    public Employee build() {
+      try {
+        Employee record = new Employee();
+        record.employeeName = fieldSetFlags()[0] ? this.employeeName : (java.lang.CharSequence) defaultValue(fields()[0]);
+        record.employeeID = fieldSetFlags()[1] ? this.employeeID : (java.lang.Long) defaultValue(fields()[1]);
+        record.age = fieldSetFlags()[2] ? this.age : (java.lang.Long) defaultValue(fields()[2]);
+        record.gender = fieldSetFlags()[3] ? this.gender : (org.apache.hadoop.hive.hbase.avro.Gender) defaultValue(fields()[3]);
+        record.contactInfo = fieldSetFlags()[4] ? this.contactInfo : (org.apache.hadoop.hive.hbase.avro.ContactInfo) defaultValue(fields()[4]);
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+  }
+}

Added: hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/EmployeeAvro.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/EmployeeAvro.java?rev=1623845&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/EmployeeAvro.java (added)
+++ hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/EmployeeAvro.java Tue Sep  9 15:18:29 2014
@@ -0,0 +1,17 @@
+/**
+ * Autogenerated by Avro
+ * 
+ * DO NOT EDIT DIRECTLY
+ */
+package org.apache.hadoop.hive.hbase.avro;
+
+@SuppressWarnings("all")
+@org.apache.avro.specific.AvroGenerated
+public interface EmployeeAvro {
+  public static final org.apache.avro.Protocol PROTOCOL = org.apache.avro.Protocol.parse("{\"protocol\":\"EmployeeAvro\",\"namespace\":\"org.apache.hadoop.hive.hbase.avro\",\"types\":[{\"type\":\"enum\",\"name\":\"Gender\",\"symbols\":[\"MALE\",\"FEMALE\"]},{\"type\":\"record\",\"name\":\"HomePhone\",\"fields\":[{\"name\":\"areaCode\",\"type\":\"long\"},{\"name\":\"number\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"OfficePhone\",\"fields\":[{\"name\":\"areaCode\",\"type\":\"long\"},{\"name\":\"number\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"Address\",\"fields\":[{\"name\":\"address1\",\"type\":\"string\"},{\"name\":\"address2\",\"type\":\"string\"},{\"name\":\"city\",\"type\":\"string\"},{\"name\":\"zipcode\",\"type\":\"long\"},{\"name\":\"county\",\"type\":[\"HomePhone\",\"OfficePhone\",\"string\",\"null\"]},{\"name\":\"aliases\",\"type\":[{\"type\":\"array\",\"items\":\"string\"},\"null\"]},{\"name\":\"metadata\",\"type\":[\"null\",{\"type\":\"map\",\"valu
 es\":\"string\"}]}]},{\"type\":\"record\",\"name\":\"ContactInfo\",\"fields\":[{\"name\":\"address\",\"type\":[{\"type\":\"array\",\"items\":\"Address\"},\"null\"]},{\"name\":\"homePhone\",\"type\":\"HomePhone\"},{\"name\":\"officePhone\",\"type\":\"OfficePhone\"}]},{\"type\":\"record\",\"name\":\"Employee\",\"fields\":[{\"name\":\"employeeName\",\"type\":\"string\"},{\"name\":\"employeeID\",\"type\":\"long\"},{\"name\":\"age\",\"type\":\"long\"},{\"name\":\"gender\",\"type\":\"Gender\"},{\"name\":\"contactInfo\",\"type\":\"ContactInfo\"}]}],\"messages\":{}}");
+
+  @SuppressWarnings("all")
+  public interface Callback extends EmployeeAvro {
+    public static final org.apache.avro.Protocol PROTOCOL = org.apache.hadoop.hive.hbase.avro.EmployeeAvro.PROTOCOL;
+  }
+}
\ No newline at end of file

Added: hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Gender.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Gender.java?rev=1623845&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Gender.java (added)
+++ hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/Gender.java Tue Sep  9 15:18:29 2014
@@ -0,0 +1,13 @@
+/**
+ * Autogenerated by Avro
+ * 
+ * DO NOT EDIT DIRECTLY
+ */
+package org.apache.hadoop.hive.hbase.avro;  
+@SuppressWarnings("all")
+@org.apache.avro.specific.AvroGenerated
+public enum Gender { 
+  MALE, FEMALE  ;
+  public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"enum\",\"name\":\"Gender\",\"namespace\":\"org.apache.hadoop.hive.hbase.avro\",\"symbols\":[\"MALE\",\"FEMALE\"]}");
+  public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; }
+}

Added: hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/HomePhone.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/HomePhone.java?rev=1623845&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/HomePhone.java (added)
+++ hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/avro/HomePhone.java Tue Sep  9 15:18:29 2014
@@ -0,0 +1,194 @@
+/**
+ * Autogenerated by Avro
+ * 
+ * DO NOT EDIT DIRECTLY
+ */
+package org.apache.hadoop.hive.hbase.avro;  
+@SuppressWarnings("all")
+@org.apache.avro.specific.AvroGenerated
+public class HomePhone extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
+  public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"HomePhone\",\"namespace\":\"org.apache.hadoop.hive.hbase.avro\",\"fields\":[{\"name\":\"areaCode\",\"type\":\"long\"},{\"name\":\"number\",\"type\":\"long\"}]}");
+  public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; }
+  @Deprecated public long areaCode;
+  @Deprecated public long number;
+
+  /**
+   * Default constructor.  Note that this does not initialize fields
+   * to their default values from the schema.  If that is desired then
+   * one should use <code>newBuilder()</code>. 
+   */
+  public HomePhone() {}
+
+  /**
+   * All-args constructor.
+   */
+  public HomePhone(java.lang.Long areaCode, java.lang.Long number) {
+    this.areaCode = areaCode;
+    this.number = number;
+  }
+
+  public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+  // Used by DatumWriter.  Applications should not call. 
+  public java.lang.Object get(int field$) {
+    switch (field$) {
+    case 0: return areaCode;
+    case 1: return number;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+  // Used by DatumReader.  Applications should not call. 
+  @SuppressWarnings(value="unchecked")
+  public void put(int field$, java.lang.Object value$) {
+    switch (field$) {
+    case 0: areaCode = (java.lang.Long)value$; break;
+    case 1: number = (java.lang.Long)value$; break;
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+
+  /**
+   * Gets the value of the 'areaCode' field.
+   */
+  public java.lang.Long getAreaCode() {
+    return areaCode;
+  }
+
+  /**
+   * Sets the value of the 'areaCode' field.
+   * @param value the value to set.
+   */
+  public void setAreaCode(java.lang.Long value) {
+    this.areaCode = value;
+  }
+
+  /**
+   * Gets the value of the 'number' field.
+   */
+  public java.lang.Long getNumber() {
+    return number;
+  }
+
+  /**
+   * Sets the value of the 'number' field.
+   * @param value the value to set.
+   */
+  public void setNumber(java.lang.Long value) {
+    this.number = value;
+  }
+
+  /** Creates a new HomePhone RecordBuilder */
+  public static org.apache.hadoop.hive.hbase.avro.HomePhone.Builder newBuilder() {
+    return new org.apache.hadoop.hive.hbase.avro.HomePhone.Builder();
+  }
+  
+  /** Creates a new HomePhone RecordBuilder by copying an existing Builder */
+  public static org.apache.hadoop.hive.hbase.avro.HomePhone.Builder newBuilder(org.apache.hadoop.hive.hbase.avro.HomePhone.Builder other) {
+    return new org.apache.hadoop.hive.hbase.avro.HomePhone.Builder(other);
+  }
+  
+  /** Creates a new HomePhone RecordBuilder by copying an existing HomePhone instance */
+  public static org.apache.hadoop.hive.hbase.avro.HomePhone.Builder newBuilder(org.apache.hadoop.hive.hbase.avro.HomePhone other) {
+    return new org.apache.hadoop.hive.hbase.avro.HomePhone.Builder(other);
+  }
+  
+  /**
+   * RecordBuilder for HomePhone instances.
+   */
+  public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase<HomePhone>
+    implements org.apache.avro.data.RecordBuilder<HomePhone> {
+
+    private long areaCode;
+    private long number;
+
+    /** Creates a new Builder */
+    private Builder() {
+      super(org.apache.hadoop.hive.hbase.avro.HomePhone.SCHEMA$);
+    }
+    
+    /** Creates a Builder by copying an existing Builder */
+    private Builder(org.apache.hadoop.hive.hbase.avro.HomePhone.Builder other) {
+      super(other);
+      if (isValidValue(fields()[0], other.areaCode)) {
+        this.areaCode = data().deepCopy(fields()[0].schema(), other.areaCode);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.number)) {
+        this.number = data().deepCopy(fields()[1].schema(), other.number);
+        fieldSetFlags()[1] = true;
+      }
+    }
+    
+    /** Creates a Builder by copying an existing HomePhone instance */
+    private Builder(org.apache.hadoop.hive.hbase.avro.HomePhone other) {
+            super(org.apache.hadoop.hive.hbase.avro.HomePhone.SCHEMA$);
+      if (isValidValue(fields()[0], other.areaCode)) {
+        this.areaCode = data().deepCopy(fields()[0].schema(), other.areaCode);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.number)) {
+        this.number = data().deepCopy(fields()[1].schema(), other.number);
+        fieldSetFlags()[1] = true;
+      }
+    }
+
+    /** Gets the value of the 'areaCode' field */
+    public java.lang.Long getAreaCode() {
+      return areaCode;
+    }
+    
+    /** Sets the value of the 'areaCode' field */
+    public org.apache.hadoop.hive.hbase.avro.HomePhone.Builder setAreaCode(long value) {
+      validate(fields()[0], value);
+      this.areaCode = value;
+      fieldSetFlags()[0] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'areaCode' field has been set */
+    public boolean hasAreaCode() {
+      return fieldSetFlags()[0];
+    }
+    
+    /** Clears the value of the 'areaCode' field */
+    public org.apache.hadoop.hive.hbase.avro.HomePhone.Builder clearAreaCode() {
+      fieldSetFlags()[0] = false;
+      return this;
+    }
+
+    /** Gets the value of the 'number' field */
+    public java.lang.Long getNumber() {
+      return number;
+    }
+    
+    /** Sets the value of the 'number' field */
+    public org.apache.hadoop.hive.hbase.avro.HomePhone.Builder setNumber(long value) {
+      validate(fields()[1], value);
+      this.number = value;
+      fieldSetFlags()[1] = true;
+      return this; 
+    }
+    
+    /** Checks whether the 'number' field has been set */
+    public boolean hasNumber() {
+      return fieldSetFlags()[1];
+    }
+    
+    /** Clears the value of the 'number' field */
+    public org.apache.hadoop.hive.hbase.avro.HomePhone.Builder clearNumber() {
+      fieldSetFlags()[1] = false;
+      return this;
+    }
+
+    @Override
+    public HomePhone build() {
+      try {
+        HomePhone record = new HomePhone();
+        record.areaCode = fieldSetFlags()[0] ? this.areaCode : (java.lang.Long) defaultValue(fields()[0]);
+        record.number = fieldSetFlags()[1] ? this.number : (java.lang.Long) defaultValue(fields()[1]);
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+  }
+}