You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2010/01/21 10:52:48 UTC

svn commit: r901625 [4/6] - in /hadoop/hive/trunk: ./ serde/src/java/org/apache/hadoop/hive/serde2/ serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/ serde/src/java/org/apache/hadoop/hive/serde2/columnar/ serde/src/java/org/apache/hadoop/hiv...

Modified: hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/dynamic_type/TestDynamicSerDe.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/dynamic_type/TestDynamicSerDe.java?rev=901625&r1=901624&r2=901625&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/dynamic_type/TestDynamicSerDe.java (original)
+++ hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/dynamic_type/TestDynamicSerDe.java Thu Jan 21 09:52:44 2010
@@ -18,33 +18,31 @@
 package org.apache.hadoop.hive.serde2.dynamic_type;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
-import java.util.Map;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 import java.util.Map.Entry;
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde.Constants;
-import org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol;
-import org.apache.hadoop.hive.serde.Constants;
-
-import junit.framework.TestCase;
 import org.apache.hadoop.io.BytesWritable;
 
 public class TestDynamicSerDe extends TestCase {
 
-  public static HashMap<String,String> makeHashMap(String... params) {
-    HashMap<String,String> r = new HashMap<String,String>(); 
-    for(int i=0; i<params.length; i+=2) {
-      r.put(params[i], params[i+1]);
+  public static HashMap<String, String> makeHashMap(String... params) {
+    HashMap<String, String> r = new HashMap<String, String>();
+    for (int i = 0; i < params.length; i += 2) {
+      r.put(params[i], params[i + 1]);
     }
     return r;
   }
+
   public void testDynamicSerDe() throws Throwable {
     try {
 
@@ -52,7 +50,7 @@
       ArrayList<String> bye = new ArrayList<String>();
       bye.add("firstString");
       bye.add("secondString");
-      HashMap<String, Integer> another = new HashMap<String, Integer>();  
+      HashMap<String, Integer> another = new HashMap<String, Integer>();
       another.put("firstKey", 1);
       another.put("secondKey", 2);
       ArrayList<Object> struct = new ArrayList<Object>();
@@ -62,25 +60,27 @@
       struct.add(Integer.valueOf(-234));
       struct.add(Double.valueOf(1.0));
       struct.add(Double.valueOf(-2.5));
-      
-      
+
       // All protocols
       ArrayList<String> protocols = new ArrayList<String>();
       ArrayList<Boolean> isBinaries = new ArrayList<Boolean>();
-      ArrayList<HashMap<String,String>> additionalParams = new ArrayList<HashMap<String,String>>();
+      ArrayList<HashMap<String, String>> additionalParams = new ArrayList<HashMap<String, String>>();
 
-      protocols.add(org.apache.hadoop.hive.serde2.thrift.TBinarySortableProtocol.class.getName());
+      protocols
+          .add(org.apache.hadoop.hive.serde2.thrift.TBinarySortableProtocol.class
+              .getName());
       isBinaries.add(true);
       additionalParams.add(makeHashMap("serialization.sort.order", "++++++"));
-      protocols.add(org.apache.hadoop.hive.serde2.thrift.TBinarySortableProtocol.class.getName());
+      protocols
+          .add(org.apache.hadoop.hive.serde2.thrift.TBinarySortableProtocol.class
+              .getName());
       isBinaries.add(true);
       additionalParams.add(makeHashMap("serialization.sort.order", "------"));
 
-
       protocols.add(org.apache.thrift.protocol.TBinaryProtocol.class.getName());
       isBinaries.add(true);
       additionalParams.add(null);
-      
+
       protocols.add(org.apache.thrift.protocol.TJSONProtocol.class.getName());
       isBinaries.add(false);
       additionalParams.add(null);
@@ -89,91 +89,103 @@
       // protocols.add(org.apache.thrift.protocol.TSimpleJSONProtocol.class.getName());
       // isBinaries.add(false);
       // additionalParams.add(null);
-      
+
       // TCTLSeparatedProtocol is not done yet.
-      protocols.add(org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class.getName());
+      protocols
+          .add(org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class
+              .getName());
       isBinaries.add(false);
       additionalParams.add(null);
-      
+
       System.out.println("input struct = " + struct);
-        
-      for(int pp = 0; pp<protocols.size(); pp++) {
-        
+
+      for (int pp = 0; pp < protocols.size(); pp++) {
+
         String protocol = protocols.get(pp);
         boolean isBinary = isBinaries.get(pp);
-        
+
         System.out.println("Testing protocol: " + protocol);
         Properties schema = new Properties();
         schema.setProperty(Constants.SERIALIZATION_FORMAT, protocol);
-        schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
-        schema.setProperty(Constants.SERIALIZATION_DDL,
-        "struct test { i32 _hello, list<string> 2bye, map<string,i32> another, i32 nhello, double d, double nd}");
-        schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe().getClass().toString());
+        schema.setProperty(
+            org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+            "test");
+        schema
+            .setProperty(
+                Constants.SERIALIZATION_DDL,
+                "struct test { i32 _hello, list<string> 2bye, map<string,i32> another, i32 nhello, double d, double nd}");
+        schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe()
+            .getClass().toString());
         HashMap<String, String> p = additionalParams.get(pp);
         if (p != null) {
-          for(Entry<String, String> e: p.entrySet()) {
+          for (Entry<String, String> e : p.entrySet()) {
             schema.setProperty(e.getKey(), e.getValue());
           }
         }
-  
+
         DynamicSerDe serde = new DynamicSerDe();
         serde.initialize(new Configuration(), schema);
-        
+
         // Try getObjectInspector
         ObjectInspector oi = serde.getObjectInspector();
         System.out.println("TypeName = " + oi.getTypeName());
 
-        
         // Try to serialize
         BytesWritable bytes = (BytesWritable) serde.serialize(struct, oi);
-        System.out.println("bytes =" + hexString(bytes)); 
-       
+        System.out.println("bytes =" + hexString(bytes));
+
         if (!isBinary) {
-          System.out.println("bytes in text =" + new String(bytes.get(), 0, bytes.getSize()));
+          System.out.println("bytes in text ="
+              + new String(bytes.get(), 0, bytes.getSize()));
         }
-        
+
         // Try to deserialize
         Object o = serde.deserialize(bytes);
         System.out.println("o class = " + o.getClass());
-        List<?> olist = (List<?>)o;
+        List<?> olist = (List<?>) o;
         System.out.println("o size = " + olist.size());
         System.out.println("o[0] class = " + olist.get(0).getClass());
         System.out.println("o[1] class = " + olist.get(1).getClass());
         System.out.println("o[2] class = " + olist.get(2).getClass());
         System.out.println("o = " + o);
-        
+
         assertEquals(struct, o);
       }
-      
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
-    
+
   }
+
   public String hexString(BytesWritable bytes) {
     StringBuilder sb = new StringBuilder();
-    for (int i=0; i<bytes.getSize(); i++) {
+    for (int i = 0; i < bytes.getSize(); i++) {
       byte b = bytes.get()[i];
-      int v = (b<0 ? 256 + b : b);
+      int v = (b < 0 ? 256 + b : b);
       sb.append(String.format("x%02x", v));
     }
     return sb.toString();
-  }  
-
+  }
 
-  private void testTBinarySortableProtocol(Object[] structs, String ddl, boolean ascending) throws Throwable{
-    int fields = ((List)structs[structs.length-1]).size();
+  private void testTBinarySortableProtocol(Object[] structs, String ddl,
+      boolean ascending) throws Throwable {
+    int fields = ((List) structs[structs.length - 1]).size();
     String order = "";
-    for(int i=0; i<fields; i++) {
-      order = order + (ascending ? "+" : "-"); 
+    for (int i = 0; i < fields; i++) {
+      order = order + (ascending ? "+" : "-");
     }
-    
+
     Properties schema = new Properties();
-    schema.setProperty(Constants.SERIALIZATION_FORMAT, org.apache.hadoop.hive.serde2.thrift.TBinarySortableProtocol.class.getName());
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
+    schema.setProperty(Constants.SERIALIZATION_FORMAT,
+        org.apache.hadoop.hive.serde2.thrift.TBinarySortableProtocol.class
+            .getName());
+    schema.setProperty(
+        org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
     schema.setProperty(Constants.SERIALIZATION_DDL, ddl);
-    schema.setProperty(Constants.SERIALIZATION_LIB, DynamicSerDe.class.getName());
+    schema.setProperty(Constants.SERIALIZATION_LIB, DynamicSerDe.class
+        .getName());
     schema.setProperty(Constants.SERIALIZATION_SORT_ORDER, order);
 
     DynamicSerDe serde = new DynamicSerDe();
@@ -183,24 +195,29 @@
 
     // Try to serialize
     BytesWritable bytes[] = new BytesWritable[structs.length];
-    for (int i=0; i<structs.length; i++) {
+    for (int i = 0; i < structs.length; i++) {
       bytes[i] = new BytesWritable();
-      BytesWritable s = (BytesWritable)serde.serialize(structs[i], oi);
+      BytesWritable s = (BytesWritable) serde.serialize(structs[i], oi);
       bytes[i].set(s);
-      if (i>0) {
-        int compareResult = bytes[i-1].compareTo(bytes[i]);
-        if ( (compareResult<0 && !ascending) || (compareResult>0 && ascending) ) {
-          System.out.println("Test failed in " + (ascending ? "ascending" : "descending") + " order.");
-          System.out.println("serialized data of " + structs[i-1] + " = " + hexString(bytes[i-1]));
-          System.out.println("serialized data of " + structs[i] + " = " + hexString(bytes[i]));
-          fail("Sort order of serialized " + structs[i-1] + " and " + structs[i] + " are reversed!");            
+      if (i > 0) {
+        int compareResult = bytes[i - 1].compareTo(bytes[i]);
+        if ((compareResult < 0 && !ascending)
+            || (compareResult > 0 && ascending)) {
+          System.out.println("Test failed in "
+              + (ascending ? "ascending" : "descending") + " order.");
+          System.out.println("serialized data of " + structs[i - 1] + " = "
+              + hexString(bytes[i - 1]));
+          System.out.println("serialized data of " + structs[i] + " = "
+              + hexString(bytes[i]));
+          fail("Sort order of serialized " + structs[i - 1] + " and "
+              + structs[i] + " are reversed!");
         }
       }
     }
 
     // Try to deserialize
     Object[] deserialized = new Object[structs.length];
-    for (int i=0; i<structs.length; i++) {
+    for (int i = 0; i < structs.length; i++) {
       deserialized[i] = serde.deserialize(bytes[i]);
       if (!structs[i].equals(deserialized[i])) {
         System.out.println("structs[i] = " + structs[i]);
@@ -210,60 +227,75 @@
       }
     }
   }
-  
+
   static int compare(Object a, Object b) {
-    if (a == null && b == null) return 0;
-    if (a == null) return -1;
-    if (b == null) return 1;
+    if (a == null && b == null) {
+      return 0;
+    }
+    if (a == null) {
+      return -1;
+    }
+    if (b == null) {
+      return 1;
+    }
     if (a instanceof List) {
-      List la = (List)a;
-      List lb = (List)b;
-      assert(la.size() == lb.size());
-      for (int i=0; i<la.size(); i++) {
+      List la = (List) a;
+      List lb = (List) b;
+      assert (la.size() == lb.size());
+      for (int i = 0; i < la.size(); i++) {
         int r = compare(la.get(i), lb.get(i));
-        if (r != 0) return r;
+        if (r != 0) {
+          return r;
+        }
       }
       return 0;
     } else if (a instanceof Number) {
       Number na = (Number) a;
       Number nb = (Number) b;
-      if (na.doubleValue() < nb.doubleValue()) return -1;
-      if (na.doubleValue() > nb.doubleValue()) return 1;
-      return 0;      
+      if (na.doubleValue() < nb.doubleValue()) {
+        return -1;
+      }
+      if (na.doubleValue() > nb.doubleValue()) {
+        return 1;
+      }
+      return 0;
     } else if (a instanceof String) {
       String sa = (String) a;
       String sb = (String) b;
       return sa.compareTo(sb);
-    } 
+    }
     return 0;
   }
-  
+
   private void sort(Object[] structs) {
-    for (int i=0; i<structs.length; i++) for (int j=i+1; j<structs.length; j++)
-      if (compare(structs[i], structs[j])>0) {
-        Object t = structs[i];
-        structs[i] = structs[j];
-        structs[j] = t;
+    for (int i = 0; i < structs.length; i++) {
+      for (int j = i + 1; j < structs.length; j++) {
+        if (compare(structs[i], structs[j]) > 0) {
+          Object t = structs[i];
+          structs[i] = structs[j];
+          structs[j] = t;
+        }
       }
-  }  
+    }
+  }
 
   public void testTBinarySortableProtocol() throws Throwable {
     try {
 
       System.out.println("Beginning Test testTBinarySortableProtocol:");
-      
+
       int num = 100;
       Random r = new Random(1234);
       Object structs[] = new Object[num];
       String ddl;
-      
+
       // Test double
-      for (int i=0; i<num; i++) {
+      for (int i = 0; i < num; i++) {
         ArrayList<Object> struct = new ArrayList<Object>();
-        if (i==0) {
+        if (i == 0) {
           struct.add(null);
         } else {
-          struct.add(Double.valueOf((r.nextDouble()-0.5)*10));
+          struct.add(Double.valueOf((r.nextDouble() - 0.5) * 10));
         }
         structs[i] = struct;
       }
@@ -274,86 +306,90 @@
       testTBinarySortableProtocol(structs, ddl, false);
 
       // Test integer
-      for (int i=0; i<num; i++) {
+      for (int i = 0; i < num; i++) {
         ArrayList<Object> struct = new ArrayList<Object>();
-        if (i==0) {
+        if (i == 0) {
           struct.add(null);
         } else {
-          struct.add((int)((r.nextDouble()-0.5)*1.5*Integer.MAX_VALUE));
+          struct.add((int) ((r.nextDouble() - 0.5) * 1.5 * Integer.MAX_VALUE));
         }
         structs[i] = struct;
       }
       sort(structs);
-      // Null should be smaller than any other value, so put a null at the front end
+      // Null should be smaller than any other value, so put a null at the front
+      // end
       // to test whether that is held.
-      ((List)structs[0]).set(0, null);
+      ((List) structs[0]).set(0, null);
       ddl = "struct test { i32 hello}";
       System.out.println("Testing " + ddl);
       testTBinarySortableProtocol(structs, ddl, true);
       testTBinarySortableProtocol(structs, ddl, false);
 
       // Test long
-      for (int i=0; i<num; i++) {
+      for (int i = 0; i < num; i++) {
         ArrayList<Object> struct = new ArrayList<Object>();
-        if (i==0) {
+        if (i == 0) {
           struct.add(null);
         } else {
-          struct.add((long)((r.nextDouble()-0.5)*1.5*Long.MAX_VALUE));
+          struct.add((long) ((r.nextDouble() - 0.5) * 1.5 * Long.MAX_VALUE));
         }
         structs[i] = struct;
       }
       sort(structs);
-      // Null should be smaller than any other value, so put a null at the front end
+      // Null should be smaller than any other value, so put a null at the front
+      // end
       // to test whether that is held.
-      ((List)structs[0]).set(0, null);
+      ((List) structs[0]).set(0, null);
       ddl = "struct test { i64 hello}";
       System.out.println("Testing " + ddl);
       testTBinarySortableProtocol(structs, ddl, true);
       testTBinarySortableProtocol(structs, ddl, false);
 
       // Test string
-      for (int i=0; i<num; i++) {
+      for (int i = 0; i < num; i++) {
         ArrayList<Object> struct = new ArrayList<Object>();
-        if (i==0) {
+        if (i == 0) {
           struct.add(null);
         } else {
-          struct.add(String.valueOf((r.nextDouble()-0.5)*1000));
+          struct.add(String.valueOf((r.nextDouble() - 0.5) * 1000));
         }
         structs[i] = struct;
       }
       sort(structs);
-      // Null should be smaller than any other value, so put a null at the front end
+      // Null should be smaller than any other value, so put a null at the front
+      // end
       // to test whether that is held.
-      ((List)structs[0]).set(0, null);
+      ((List) structs[0]).set(0, null);
       ddl = "struct test { string hello}";
       System.out.println("Testing " + ddl);
       testTBinarySortableProtocol(structs, ddl, true);
       testTBinarySortableProtocol(structs, ddl, false);
 
       // Test string + double
-      for (int i=0; i<num; i++) {
+      for (int i = 0; i < num; i++) {
         ArrayList<Object> struct = new ArrayList<Object>();
-        if (i%9==0) {
+        if (i % 9 == 0) {
           struct.add(null);
         } else {
-          struct.add("str" + (i/5));
+          struct.add("str" + (i / 5));
         }
-        if (i%7==0) {
+        if (i % 7 == 0) {
           struct.add(null);
         } else {
-          struct.add(Double.valueOf((r.nextDouble()-0.5)*10));
+          struct.add(Double.valueOf((r.nextDouble() - 0.5) * 10));
         }
         structs[i] = struct;
       }
       sort(structs);
-      // Null should be smaller than any other value, so put a null at the front end
+      // Null should be smaller than any other value, so put a null at the front
+      // end
       // to test whether that is held.
-      ((List)structs[0]).set(0, null);
+      ((List) structs[0]).set(0, null);
       ddl = "struct test { string hello, double another}";
       System.out.println("Testing " + ddl);
       testTBinarySortableProtocol(structs, ddl, true);
       testTBinarySortableProtocol(structs, ddl, false);
-      
+
       System.out.println("Test testTBinarySortableProtocol passed!");
     } catch (Throwable e) {
       e.printStackTrace();
@@ -361,16 +397,14 @@
     }
   }
 
-
   public void testConfigurableTCTLSeparated() throws Throwable {
     try {
 
-
       // Try to construct an object
       ArrayList<String> bye = new ArrayList<String>();
       bye.add("firstString");
       bye.add("secondString");
-      HashMap<String, Integer> another = new HashMap<String, Integer>();  
+      HashMap<String, Integer> another = new HashMap<String, Integer>();
       another.put("firstKey", 1);
       another.put("secondKey", 2);
       ArrayList<Object> struct = new ArrayList<Object>();
@@ -379,11 +413,17 @@
       struct.add(another);
 
       Properties schema = new Properties();
-      schema.setProperty(Constants.SERIALIZATION_FORMAT, org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class.getName());
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
-      schema.setProperty(Constants.SERIALIZATION_DDL,
-                         "struct test { i32 hello, list<string> bye, map<string,i32> another}");
-      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe().getClass().toString());
+      schema.setProperty(Constants.SERIALIZATION_FORMAT,
+          org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class
+              .getName());
+      schema.setProperty(
+          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+          "test");
+      schema
+          .setProperty(Constants.SERIALIZATION_DDL,
+              "struct test { i32 hello, list<string> bye, map<string,i32> another}");
+      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe()
+          .getClass().toString());
 
       schema.setProperty(Constants.FIELD_DELIM, "9");
       schema.setProperty(Constants.COLLECTION_DELIM, "1");
@@ -393,42 +433,44 @@
       DynamicSerDe serde = new DynamicSerDe();
       serde.initialize(new Configuration(), schema);
 
-      TCTLSeparatedProtocol prot = (TCTLSeparatedProtocol)serde.oprot_;
+      TCTLSeparatedProtocol prot = (TCTLSeparatedProtocol) serde.oprot_;
       assertTrue(prot.getPrimarySeparator().equals("\u0009"));
-      
+
       ObjectInspector oi = serde.getObjectInspector();
 
       // Try to serialize
       BytesWritable bytes = (BytesWritable) serde.serialize(struct, oi);
-        
+
       hexString(bytes);
 
-      String compare = "234" + "\u0009" + "firstString" + "\u0001" + "secondString" + "\u0009" + "firstKey" + "\u0004" + "1" + "\u0001" + "secondKey" + "\u0004" + "2";
+      String compare = "234" + "\u0009" + "firstString" + "\u0001"
+          + "secondString" + "\u0009" + "firstKey" + "\u0004" + "1" + "\u0001"
+          + "secondKey" + "\u0004" + "2";
 
-      System.out.println("bytes in text =" + new String(bytes.get(), 0, bytes.getSize()) + ">");
+      System.out.println("bytes in text ="
+          + new String(bytes.get(), 0, bytes.getSize()) + ">");
       System.out.println("compare to    =" + compare + ">");
-        
-      assertTrue(compare.equals( new String(bytes.get(), 0, bytes.getSize())));
-      
+
+      assertTrue(compare.equals(new String(bytes.get(), 0, bytes.getSize())));
+
       // Try to deserialize
       Object o = serde.deserialize(bytes);
       System.out.println("o class = " + o.getClass());
-      List<?> olist = (List<?>)o;
+      List<?> olist = (List<?>) o;
       System.out.println("o size = " + olist.size());
       System.out.println("o[0] class = " + olist.get(0).getClass());
       System.out.println("o[1] class = " + olist.get(1).getClass());
       System.out.println("o[2] class = " + olist.get(2).getClass());
       System.out.println("o = " + o);
-        
+
       assertEquals(o, struct);
-      
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
-    
-  }  
 
+  }
 
   /**
    * Tests a single null list within a struct with return nulls on
@@ -437,10 +479,9 @@
   public void testNulls1() throws Throwable {
     try {
 
-
       // Try to construct an object
       ArrayList<String> bye = null;
-      HashMap<String, Integer> another = new HashMap<String, Integer>();  
+      HashMap<String, Integer> another = new HashMap<String, Integer>();
       another.put("firstKey", 1);
       another.put("secondKey", 2);
       ArrayList<Object> struct = new ArrayList<Object>();
@@ -449,33 +490,39 @@
       struct.add(another);
 
       Properties schema = new Properties();
-      schema.setProperty(Constants.SERIALIZATION_FORMAT, org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class.getName());
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
-      schema.setProperty(Constants.SERIALIZATION_DDL,
-                         "struct test { i32 hello, list<string> bye, map<string,i32> another}");
-      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe().getClass().toString());
+      schema.setProperty(Constants.SERIALIZATION_FORMAT,
+          org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class
+              .getName());
+      schema.setProperty(
+          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+          "test");
+      schema
+          .setProperty(Constants.SERIALIZATION_DDL,
+              "struct test { i32 hello, list<string> bye, map<string,i32> another}");
+      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe()
+          .getClass().toString());
       schema.setProperty(TCTLSeparatedProtocol.ReturnNullsKey, "true");
 
       DynamicSerDe serde = new DynamicSerDe();
       serde.initialize(new Configuration(), schema);
 
       ObjectInspector oi = serde.getObjectInspector();
-      
+
       // Try to serialize
       BytesWritable bytes = (BytesWritable) serde.serialize(struct, oi);
-        
+
       hexString(bytes);
 
       // Try to deserialize
       Object o = serde.deserialize(bytes);
       assertEquals(struct, o);
-      
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
-    
-  }  
+
+  }
 
   /**
    * Tests all elements of a struct being null with return nulls on
@@ -484,7 +531,6 @@
   public void testNulls2() throws Throwable {
     try {
 
-
       // Try to construct an object
       ArrayList<String> bye = null;
       HashMap<String, Integer> another = null;
@@ -494,11 +540,17 @@
       struct.add(another);
 
       Properties schema = new Properties();
-      schema.setProperty(Constants.SERIALIZATION_FORMAT, org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class.getName());
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
-      schema.setProperty(Constants.SERIALIZATION_DDL,
-                         "struct test { i32 hello, list<string> bye, map<string,i32> another}");
-      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe().getClass().toString());
+      schema.setProperty(Constants.SERIALIZATION_FORMAT,
+          org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class
+              .getName());
+      schema.setProperty(
+          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+          "test");
+      schema
+          .setProperty(Constants.SERIALIZATION_DDL,
+              "struct test { i32 hello, list<string> bye, map<string,i32> another}");
+      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe()
+          .getClass().toString());
       schema.setProperty(TCTLSeparatedProtocol.ReturnNullsKey, "true");
 
       DynamicSerDe serde = new DynamicSerDe();
@@ -508,26 +560,27 @@
 
       // Try to serialize
       BytesWritable bytes = (BytesWritable) serde.serialize(struct, oi);
-        
+
       hexString(bytes);
 
       // Try to deserialize
       Object o = serde.deserialize(bytes);
-      List<?> olist = (List<?>)o;
+      List<?> olist = (List<?>) o;
 
       assertTrue(olist.size() == 3);
       assertEquals(null, olist.get(0));
       assertEquals(null, olist.get(1));
       assertEquals(null, olist.get(2));
-      
-      //      assertEquals(o, struct); Cannot do this because types of null lists are wrong.
-      
+
+      // assertEquals(o, struct); Cannot do this because types of null lists are
+      // wrong.
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
-    
-  }  
+
+  }
 
   /**
    * Tests map and list being empty with return nulls on
@@ -536,9 +589,8 @@
   public void testNulls3() throws Throwable {
     try {
 
-
       // Try to construct an object
-      ArrayList<String> bye = new ArrayList<String> ();
+      ArrayList<String> bye = new ArrayList<String>();
       HashMap<String, Integer> another = null;
       ArrayList<Object> struct = new ArrayList<Object>();
       struct.add(null);
@@ -546,11 +598,17 @@
       struct.add(another);
 
       Properties schema = new Properties();
-      schema.setProperty(Constants.SERIALIZATION_FORMAT, org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class.getName());
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
-      schema.setProperty(Constants.SERIALIZATION_DDL,
-                         "struct test { i32 hello, list<string> bye, map<string,i32> another}");
-      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe().getClass().toString());
+      schema.setProperty(Constants.SERIALIZATION_FORMAT,
+          org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class
+              .getName());
+      schema.setProperty(
+          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+          "test");
+      schema
+          .setProperty(Constants.SERIALIZATION_DDL,
+              "struct test { i32 hello, list<string> bye, map<string,i32> another}");
+      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe()
+          .getClass().toString());
 
       schema.setProperty(TCTLSeparatedProtocol.ReturnNullsKey, "true");
       DynamicSerDe serde = new DynamicSerDe();
@@ -560,27 +618,27 @@
 
       // Try to serialize
       BytesWritable bytes = (BytesWritable) serde.serialize(struct, oi);
-        
+
       hexString(bytes);
 
       // Try to deserialize
       Object o = serde.deserialize(bytes);
-      List<?> olist = (List<?>)o;
+      List<?> olist = (List<?>) o;
 
       assertTrue(olist.size() == 3);
       assertEquals(null, olist.get(0));
-      assertEquals(0, ((List<?>)olist.get(1)).size());
+      assertEquals(0, ((List<?>) olist.get(1)).size());
       assertEquals(null, olist.get(2));
-      
-      //      assertEquals(o, struct); Cannot do this because types of null lists are wrong.
-      
+
+      // assertEquals(o, struct); Cannot do this because types of null lists are
+      // wrong.
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
-    
-  }  
 
+  }
 
   /**
    * Tests map and list null/empty with return nulls *off*
@@ -589,9 +647,8 @@
   public void testNulls4() throws Throwable {
     try {
 
-
       // Try to construct an object
-      ArrayList<String> bye = new ArrayList<String> ();
+      ArrayList<String> bye = new ArrayList<String>();
       HashMap<String, Integer> another = null;
       ArrayList<Object> struct = new ArrayList<Object>();
       struct.add(null);
@@ -599,11 +656,17 @@
       struct.add(another);
 
       Properties schema = new Properties();
-      schema.setProperty(Constants.SERIALIZATION_FORMAT, org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class.getName());
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
-      schema.setProperty(Constants.SERIALIZATION_DDL,
-                         "struct test { i32 hello, list<string> bye, map<string,i32> another}");
-      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe().getClass().toString());
+      schema.setProperty(Constants.SERIALIZATION_FORMAT,
+          org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class
+              .getName());
+      schema.setProperty(
+          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+          "test");
+      schema
+          .setProperty(Constants.SERIALIZATION_DDL,
+              "struct test { i32 hello, list<string> bye, map<string,i32> another}");
+      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe()
+          .getClass().toString());
 
       schema.setProperty(TCTLSeparatedProtocol.ReturnNullsKey, "false");
       DynamicSerDe serde = new DynamicSerDe();
@@ -613,29 +676,29 @@
 
       // Try to serialize
       BytesWritable bytes = (BytesWritable) serde.serialize(struct, oi);
-        
+
       hexString(bytes);
 
       // Try to deserialize
       Object o = serde.deserialize(bytes);
-      List<?> olist = (List<?>)o;
+      List<?> olist = (List<?>) o;
 
       assertTrue(olist.size() == 3);
-      assertEquals(new Integer(0), (Integer)olist.get(0));
-      List<?> num1 = (List<?>)olist.get(1);
+      assertEquals(new Integer(0), (Integer) olist.get(0));
+      List<?> num1 = (List<?>) olist.get(1);
       assertTrue(num1.size() == 0);
-      Map<?,?> num2 = (Map<?,?>)olist.get(2);
+      Map<?, ?> num2 = (Map<?, ?>) olist.get(2);
       assertTrue(num2.size() == 0);
-      
-      //      assertEquals(o, struct); Cannot do this because types of null lists are wrong.
-      
+
+      // assertEquals(o, struct); Cannot do this because types of null lists are
+      // wrong.
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
-    
-  }  
 
+  }
 
   /**
    * Tests map and list null/empty with return nulls *off*
@@ -644,16 +707,21 @@
   public void testStructsinStructs() throws Throwable {
     try {
 
-
       Properties schema = new Properties();
-      //      schema.setProperty(Constants.SERIALIZATION_FORMAT, org.apache.thrift.protocol.TJSONProtocol.class.getName());
-      schema.setProperty(Constants.SERIALIZATION_FORMAT, org.apache.thrift.protocol.TBinaryProtocol.class.getName());
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
-      schema.setProperty(Constants.SERIALIZATION_DDL,
-                         "struct inner { i32 field1, string field2 },struct  test {inner foo,  i32 hello, list<string> bye, map<string,i32> another}");
-      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe().getClass().toString());
+      // schema.setProperty(Constants.SERIALIZATION_FORMAT,
+      // org.apache.thrift.protocol.TJSONProtocol.class.getName());
+      schema.setProperty(Constants.SERIALIZATION_FORMAT,
+          org.apache.thrift.protocol.TBinaryProtocol.class.getName());
+      schema.setProperty(
+          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+          "test");
+      schema
+          .setProperty(
+              Constants.SERIALIZATION_DDL,
+              "struct inner { i32 field1, string field2 },struct  test {inner foo,  i32 hello, list<string> bye, map<string,i32> another}");
+      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe()
+          .getClass().toString());
 
-      
       //
       // construct object of above type
       //
@@ -664,10 +732,10 @@
       innerStruct.add(new String("hello world"));
 
       // construct outer struct
-      ArrayList<String> bye = new ArrayList<String> ();
+      ArrayList<String> bye = new ArrayList<String>();
       bye.add("firstString");
       bye.add("secondString");
-      HashMap<String, Integer> another = new HashMap<String, Integer>();  
+      HashMap<String, Integer> another = new HashMap<String, Integer>();
       another.put("firstKey", 1);
       another.put("secondKey", 2);
 
@@ -688,8 +756,7 @@
 
       // Try to deserialize
       Object o = serde.deserialize(bytes);
-      List<?> olist = (List<?>)o;
-
+      List<?> olist = (List<?>) o;
 
       assertEquals(4, olist.size());
       assertEquals(innerStruct, olist.get(0));
@@ -701,11 +768,8 @@
       e.printStackTrace();
       throw e;
     }
-    
-  }  
-
-
 
+  }
 
   public void testSkip() throws Throwable {
     try {
@@ -714,7 +778,7 @@
       ArrayList<String> bye = new ArrayList<String>();
       bye.add("firstString");
       bye.add("secondString");
-      HashMap<String, Integer> another = new HashMap<String, Integer>();  
+      HashMap<String, Integer> another = new HashMap<String, Integer>();
       another.put("firstKey", 1);
       another.put("secondKey", 2);
       ArrayList<Object> struct = new ArrayList<Object>();
@@ -723,11 +787,17 @@
       struct.add(another);
 
       Properties schema = new Properties();
-      schema.setProperty(Constants.SERIALIZATION_FORMAT, org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class.getName());
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME, "test");
-      schema.setProperty(Constants.SERIALIZATION_DDL,
-                         "struct test { i32 hello, list<string> bye, map<string,i32> another}");
-      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe().getClass().toString());
+      schema.setProperty(Constants.SERIALIZATION_FORMAT,
+          org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol.class
+              .getName());
+      schema.setProperty(
+          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+          "test");
+      schema
+          .setProperty(Constants.SERIALIZATION_DDL,
+              "struct test { i32 hello, list<string> bye, map<string,i32> another}");
+      schema.setProperty(Constants.SERIALIZATION_LIB, new DynamicSerDe()
+          .getClass().toString());
 
       schema.setProperty(Constants.FIELD_DELIM, "9");
       schema.setProperty(Constants.COLLECTION_DELIM, "1");
@@ -737,47 +807,51 @@
       DynamicSerDe serde = new DynamicSerDe();
       serde.initialize(new Configuration(), schema);
 
-      TCTLSeparatedProtocol prot = (TCTLSeparatedProtocol)serde.oprot_;
+      TCTLSeparatedProtocol prot = (TCTLSeparatedProtocol) serde.oprot_;
       assertTrue(prot.getPrimarySeparator().equals("\u0009"));
-      
+
       ObjectInspector oi = serde.getObjectInspector();
 
       // Try to serialize
       BytesWritable bytes = (BytesWritable) serde.serialize(struct, oi);
-        
+
       hexString(bytes);
 
-      String compare = "234" + "\u0009" + "firstString" + "\u0001" + "secondString" + "\u0009" + "firstKey" + "\u0004" + "1" + "\u0001" + "secondKey" + "\u0004" + "2";
+      String compare = "234" + "\u0009" + "firstString" + "\u0001"
+          + "secondString" + "\u0009" + "firstKey" + "\u0004" + "1" + "\u0001"
+          + "secondKey" + "\u0004" + "2";
 
-      System.out.println("bytes in text =" + new String(bytes.get(), 0, bytes.getSize()) + ">");
+      System.out.println("bytes in text ="
+          + new String(bytes.get(), 0, bytes.getSize()) + ">");
       System.out.println("compare to    =" + compare + ">");
-        
-      assertTrue(compare.equals( new String(bytes.get(), 0, bytes.getSize())));
-      
-      schema.setProperty(Constants.SERIALIZATION_DDL,
-                         "struct test { i32 hello, skip list<string> bye, map<string,i32> another}");
+
+      assertTrue(compare.equals(new String(bytes.get(), 0, bytes.getSize())));
+
+      schema
+          .setProperty(Constants.SERIALIZATION_DDL,
+              "struct test { i32 hello, skip list<string> bye, map<string,i32> another}");
 
       serde.initialize(new Configuration(), schema);
 
       // Try to deserialize
       Object o = serde.deserialize(bytes);
       System.out.println("o class = " + o.getClass());
-      List<?> olist = (List<?>)o;
+      List<?> olist = (List<?>) o;
       System.out.println("o size = " + olist.size());
       System.out.println("o = " + o);
-        
+
       assertEquals(null, olist.get(1));
 
       // set the skipped field to null
-      struct.set(1,null);
+      struct.set(1, null);
 
       assertEquals(o, struct);
-      
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
-    
-  }  
+
+  }
 
 }

Modified: hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyArrayMapStruct.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyArrayMapStruct.java?rev=901625&r1=901624&r2=901625&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyArrayMapStruct.java (original)
+++ hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyArrayMapStruct.java Thu Jan 21 09:52:44 2010
@@ -17,24 +17,19 @@
  */
 package org.apache.hadoop.hive.serde2.lazy;
 
-
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
+
+import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
-import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazyObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
-import java.util.HashMap;
-
-import junit.framework.TestCase;
 
 public class TestLazyArrayMapStruct extends TestCase {
 
@@ -45,51 +40,63 @@
     try {
       // Array of Byte
       Text nullSequence = new Text("\\N");
-      ObjectInspector oi = LazyFactory.createLazyObjectInspector(
-          TypeInfoUtils.getTypeInfosFromTypeString("array<tinyint>").get(0), 
-          new byte[]{(byte)1}, 0, nullSequence, false, (byte)0);
-      LazyArray b = (LazyArray)LazyFactory.createLazyObject(oi);
-      byte[] data = new byte[]{'-', '1', 1, '\\', 'N', 1, '8'};
+      ObjectInspector oi = LazyFactory.createLazyObjectInspector(TypeInfoUtils
+          .getTypeInfosFromTypeString("array<tinyint>").get(0),
+          new byte[] { (byte) 1 }, 0, nullSequence, false, (byte) 0);
+      LazyArray b = (LazyArray) LazyFactory.createLazyObject(oi);
+      byte[] data = new byte[] { '-', '1', 1, '\\', 'N', 1, '8' };
       TestLazyPrimitive.initLazyObject(b, data, 0, data.length);
-      
+
       assertNull(b.getListElementObject(-1));
-      assertEquals(new ByteWritable((byte)-1), ((LazyByte)b.getListElementObject(0)).getWritableObject());
-      assertEquals(new ByteWritable((byte)-1), ((LazyByte)b.getList().get(0)).getWritableObject());
+      assertEquals(new ByteWritable((byte) -1), ((LazyByte) b
+          .getListElementObject(0)).getWritableObject());
+      assertEquals(new ByteWritable((byte) -1), ((LazyByte) b.getList().get(0))
+          .getWritableObject());
       assertNull(b.getListElementObject(1));
       assertNull(b.getList().get(1));
-      assertEquals(new ByteWritable((byte)8), ((LazyByte)b.getListElementObject(2)).getWritableObject());
-      assertEquals(new ByteWritable((byte)8), ((LazyByte)b.getList().get(2)).getWritableObject());
+      assertEquals(new ByteWritable((byte) 8), ((LazyByte) b
+          .getListElementObject(2)).getWritableObject());
+      assertEquals(new ByteWritable((byte) 8), ((LazyByte) b.getList().get(2))
+          .getWritableObject());
       assertNull(b.getListElementObject(3));
       assertEquals(3, b.getList().size());
-      
+
       // Array of String
-      oi = LazyFactory.createLazyObjectInspector(
-          TypeInfoUtils.getTypeInfosFromTypeString("array<string>").get(0), 
-          new byte[]{(byte)'\t'}, 0, nullSequence, false, (byte)0);
-      b = (LazyArray)LazyFactory.createLazyObject(oi);
-      data = new byte[]{'a', 'b', '\t', 'c', '\t', '\\', 'N', '\t', '\t', 'd'};
+      oi = LazyFactory.createLazyObjectInspector(TypeInfoUtils
+          .getTypeInfosFromTypeString("array<string>").get(0),
+          new byte[] { (byte) '\t' }, 0, nullSequence, false, (byte) 0);
+      b = (LazyArray) LazyFactory.createLazyObject(oi);
+      data = new byte[] { 'a', 'b', '\t', 'c', '\t', '\\', 'N', '\t', '\t', 'd' };
       // Note: the first and last element of the byte[] are NOT used
       TestLazyPrimitive.initLazyObject(b, data, 1, data.length - 2);
       assertNull(b.getListElementObject(-1));
-      assertEquals(new Text("b"), ((LazyString)b.getListElementObject(0)).getWritableObject());
-      assertEquals(new Text("b"), ((LazyString)b.getList().get(0)).getWritableObject());
-      assertEquals(new Text("c"), ((LazyString)b.getListElementObject(1)).getWritableObject());
-      assertEquals(new Text("c"), ((LazyString)b.getList().get(1)).getWritableObject());
-      assertNull(((LazyString)b.getListElementObject(2)));
-      assertNull(((LazyString)b.getList().get(2)));
-      assertEquals(new Text(""), ((LazyString)b.getListElementObject(3)).getWritableObject());
-      assertEquals(new Text(""), ((LazyString)b.getList().get(3)).getWritableObject());
-      assertEquals(new Text(""), ((LazyString)b.getListElementObject(4)).getWritableObject());
-      assertEquals(new Text(""), ((LazyString)b.getList().get(4)).getWritableObject());
-      assertNull(((LazyString)b.getListElementObject(5)));
+      assertEquals(new Text("b"), ((LazyString) b.getListElementObject(0))
+          .getWritableObject());
+      assertEquals(new Text("b"), ((LazyString) b.getList().get(0))
+          .getWritableObject());
+      assertEquals(new Text("c"), ((LazyString) b.getListElementObject(1))
+          .getWritableObject());
+      assertEquals(new Text("c"), ((LazyString) b.getList().get(1))
+          .getWritableObject());
+      assertNull((b.getListElementObject(2)));
+      assertNull((b.getList().get(2)));
+      assertEquals(new Text(""), ((LazyString) b.getListElementObject(3))
+          .getWritableObject());
+      assertEquals(new Text(""), ((LazyString) b.getList().get(3))
+          .getWritableObject());
+      assertEquals(new Text(""), ((LazyString) b.getListElementObject(4))
+          .getWritableObject());
+      assertEquals(new Text(""), ((LazyString) b.getList().get(4))
+          .getWritableObject());
+      assertNull((b.getListElementObject(5)));
       assertEquals(5, b.getList().size());
-      
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
   }
-    
+
   /**
    * Test the LazyMap class.
    */
@@ -98,114 +105,140 @@
       {
         // Map of Integer to String
         Text nullSequence = new Text("\\N");
-        ObjectInspector oi = LazyFactory.createLazyObjectInspector(
-            TypeInfoUtils.getTypeInfosFromTypeString("map<int,string>").get(0), 
-            new byte[]{(byte)1, (byte)2}, 0, nullSequence, false, (byte)0);
-        LazyMap b = (LazyMap)LazyFactory.createLazyObject(oi);
-        byte[] data = new byte[]{'2', 2, 'd', 'e', 'f', 1, '-', '1', 2, '\\', 'N', 1, '0', 2, '0', 1, '8', 2, 'a', 'b', 'c'};
+        ObjectInspector oi = LazyFactory
+            .createLazyObjectInspector(TypeInfoUtils
+                .getTypeInfosFromTypeString("map<int,string>").get(0),
+                new byte[] { (byte) 1, (byte) 2 }, 0, nullSequence, false,
+                (byte) 0);
+        LazyMap b = (LazyMap) LazyFactory.createLazyObject(oi);
+        byte[] data = new byte[] { '2', 2, 'd', 'e', 'f', 1, '-', '1', 2, '\\',
+            'N', 1, '0', 2, '0', 1, '8', 2, 'a', 'b', 'c' };
         TestLazyPrimitive.initLazyObject(b, data, 0, data.length);
-        
-        assertEquals(new Text("def"), ((LazyString)b.getMapValueElement(new IntWritable(2))).getWritableObject());
+
+        assertEquals(new Text("def"), ((LazyString) b
+            .getMapValueElement(new IntWritable(2))).getWritableObject());
         assertNull(b.getMapValueElement(new IntWritable(-1)));
-        assertEquals(new Text("0"), ((LazyString)b.getMapValueElement(new IntWritable(0))).getWritableObject());
-        assertEquals(new Text("abc"), ((LazyString)b.getMapValueElement(new IntWritable(8))).getWritableObject());
+        assertEquals(new Text("0"), ((LazyString) b
+            .getMapValueElement(new IntWritable(0))).getWritableObject());
+        assertEquals(new Text("abc"), ((LazyString) b
+            .getMapValueElement(new IntWritable(8))).getWritableObject());
         assertNull(b.getMapValueElement(new IntWritable(12345)));
-        
+
         assertEquals("{2:'def',-1:null,0:'0',8:'abc'}".replace('\'', '\"'),
             SerDeUtils.getJSONString(b, oi));
       }
-      
+
       {
         // Map of String to String
         Text nullSequence = new Text("\\N");
         ObjectInspector oi = LazyFactory.createLazyObjectInspector(
-            TypeInfoUtils.getTypeInfosFromTypeString("map<string,string>").get(0), 
-            new byte[]{(byte)'#', (byte)'\t'}, 0, nullSequence, false, (byte)0);
-        LazyMap b = (LazyMap)LazyFactory.createLazyObject(oi);
-        byte[] data = new byte[]{'2', '\t', 'd', '\t', 'f', '#', '2', '\t', 'd', '#', '-', '1', '#', '0', '\t', '0', '#', '8', '\t', 'a', 'b', 'c'};
+            TypeInfoUtils.getTypeInfosFromTypeString("map<string,string>").get(
+                0), new byte[] { (byte) '#', (byte) '\t' }, 0, nullSequence,
+            false, (byte) 0);
+        LazyMap b = (LazyMap) LazyFactory.createLazyObject(oi);
+        byte[] data = new byte[] { '2', '\t', 'd', '\t', 'f', '#', '2', '\t',
+            'd', '#', '-', '1', '#', '0', '\t', '0', '#', '8', '\t', 'a', 'b',
+            'c' };
         TestLazyPrimitive.initLazyObject(b, data, 0, data.length);
-        
-        assertEquals(new Text("d\tf"), ((LazyString)b.getMapValueElement(new Text("2"))).getWritableObject());
+
+        assertEquals(new Text("d\tf"), ((LazyString) b
+            .getMapValueElement(new Text("2"))).getWritableObject());
         assertNull(b.getMapValueElement(new Text("-1")));
-        assertEquals(new Text("0"), ((LazyString)b.getMapValueElement(new Text("0"))).getWritableObject());
-        assertEquals(new Text("abc"), ((LazyString)b.getMapValueElement(new Text("8"))).getWritableObject());
+        assertEquals(new Text("0"), ((LazyString) b
+            .getMapValueElement(new Text("0"))).getWritableObject());
+        assertEquals(new Text("abc"), ((LazyString) b
+            .getMapValueElement(new Text("8"))).getWritableObject());
         assertNull(b.getMapValueElement(new Text("-")));
-        
-        assertEquals("{'2':'d\\tf','2':'d','-1':null,'0':'0','8':'abc'}".replace('\'', '\"'),
-            SerDeUtils.getJSONString(b, oi));
+
+        assertEquals("{'2':'d\\tf','2':'d','-1':null,'0':'0','8':'abc'}"
+            .replace('\'', '\"'), SerDeUtils.getJSONString(b, oi));
       }
-      
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
   }
-  
+
   /**
    * Test the LazyStruct class.
    */
   public void testLazyStruct() throws Throwable {
     try {
       {
-        ArrayList<TypeInfo> fieldTypeInfos = 
-          TypeInfoUtils.getTypeInfosFromTypeString("int,array<string>,map<string,string>,string");
-        List<String> fieldNames = Arrays.asList(new String[]{"a", "b", "c", "d"});
+        ArrayList<TypeInfo> fieldTypeInfos = TypeInfoUtils
+            .getTypeInfosFromTypeString("int,array<string>,map<string,string>,string");
+        List<String> fieldNames = Arrays.asList(new String[] { "a", "b", "c",
+            "d" });
         Text nullSequence = new Text("\\N");
-        
+
         ObjectInspector oi = LazyFactory.createLazyStructInspector(fieldNames,
-            fieldTypeInfos, new byte[] {' ', ':', '='}, nullSequence, false, false, (byte)0);
-        LazyStruct o = (LazyStruct)LazyFactory.createLazyObject(oi);
-        
+            fieldTypeInfos, new byte[] { ' ', ':', '=' }, nullSequence, false,
+            false, (byte) 0);
+        LazyStruct o = (LazyStruct) LazyFactory.createLazyObject(oi);
+
         Text data;
-        
+
         data = new Text("123 a:b:c d=e:f=g hi");
-        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data.getLength());
-        assertEquals("{'a':123,'b':['a','b','c'],'c':{'d':'e','f':'g'},'d':'hi'}".replace("'", "\""),
-            SerDeUtils.getJSONString(o, oi));
+        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data
+            .getLength());
+        assertEquals(
+            "{'a':123,'b':['a','b','c'],'c':{'d':'e','f':'g'},'d':'hi'}"
+                .replace("'", "\""), SerDeUtils.getJSONString(o, oi));
 
         data = new Text("123 \\N d=e:f=g \\N");
-        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data.getLength());
-        assertEquals("{'a':123,'b':null,'c':{'d':'e','f':'g'},'d':null}".replace("'", "\""),
-            SerDeUtils.getJSONString(o, oi));
+        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data
+            .getLength());
+        assertEquals("{'a':123,'b':null,'c':{'d':'e','f':'g'},'d':null}"
+            .replace("'", "\""), SerDeUtils.getJSONString(o, oi));
 
         data = new Text("\\N a d=\\N:f=g:h no tail");
-        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data.getLength());
-        assertEquals("{'a':null,'b':['a'],'c':{'d':null,'f':'g','h':null},'d':'no'}".replace("'", "\""),
-            SerDeUtils.getJSONString(o, oi));
+        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data
+            .getLength());
+        assertEquals(
+            "{'a':null,'b':['a'],'c':{'d':null,'f':'g','h':null},'d':'no'}"
+                .replace("'", "\""), SerDeUtils.getJSONString(o, oi));
 
         data = new Text("\\N :a:: \\N no tail");
-        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data.getLength());
-        assertEquals("{'a':null,'b':['','a','',''],'c':null,'d':'no'}".replace("'", "\""),
-            SerDeUtils.getJSONString(o, oi));
+        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data
+            .getLength());
+        assertEquals("{'a':null,'b':['','a','',''],'c':null,'d':'no'}".replace(
+            "'", "\""), SerDeUtils.getJSONString(o, oi));
 
         data = new Text("123   ");
-        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data.getLength());
+        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data
+            .getLength());
         assertEquals("{'a':123,'b':[],'c':{},'d':''}".replace("'", "\""),
             SerDeUtils.getJSONString(o, oi));
 
         data = new Text(": : : :");
-        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data.getLength());
-        assertEquals("{'a':null,'b':['',''],'c':{'':null,'':null},'d':':'}".replace("'", "\""),
-            SerDeUtils.getJSONString(o, oi));
+        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data
+            .getLength());
+        assertEquals("{'a':null,'b':['',''],'c':{'':null,'':null},'d':':'}"
+            .replace("'", "\""), SerDeUtils.getJSONString(o, oi));
 
         data = new Text("= = = =");
-        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data.getLength());
-        assertEquals("{'a':null,'b':['='],'c':{'':''},'d':'='}".replace("'", "\""),
-            SerDeUtils.getJSONString(o, oi));
-        
+        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data
+            .getLength());
+        assertEquals("{'a':null,'b':['='],'c':{'':''},'d':'='}".replace("'",
+            "\""), SerDeUtils.getJSONString(o, oi));
+
         // test LastColumnTakesRest
-        oi = LazyFactory.createLazyStructInspector(Arrays.asList(new String[]{"a","b","c","d"}),
-            fieldTypeInfos, new byte[] {' ', ':', '='}, nullSequence, true, false, (byte)0);
-        o = (LazyStruct)LazyFactory.createLazyObject(oi);
+        oi = LazyFactory.createLazyStructInspector(Arrays.asList(new String[] {
+            "a", "b", "c", "d" }), fieldTypeInfos,
+            new byte[] { ' ', ':', '=' }, nullSequence, true, false, (byte) 0);
+        o = (LazyStruct) LazyFactory.createLazyObject(oi);
         data = new Text("\\N a d=\\N:f=g:h has tail");
-        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data.getLength());
-        assertEquals("{'a':null,'b':['a'],'c':{'d':null,'f':'g','h':null},'d':'has tail'}".replace("'", "\""),
-            SerDeUtils.getJSONString(o, oi));
+        TestLazyPrimitive.initLazyObject(o, data.getBytes(), 0, data
+            .getLength());
+        assertEquals(
+            "{'a':null,'b':['a'],'c':{'d':null,'f':'g','h':null},'d':'has tail'}"
+                .replace("'", "\""), SerDeUtils.getJSONString(o, oi));
       }
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
   }
-  
+
 }

Modified: hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyPrimitive.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyPrimitive.java?rev=901625&r1=901624&r2=901625&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyPrimitive.java (original)
+++ hadoop/hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyPrimitive.java Thu Jan 21 09:52:44 2010
@@ -17,110 +17,112 @@
  */
 package org.apache.hadoop.hive.serde2.lazy;
 
+import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.serde2.ByteStream;
 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.objectinspector.primitive.LazyPrimitiveObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
-
 public class TestLazyPrimitive extends TestCase {
 
   /**
-   * Initialize the LazyObject with the parameters, wrapping the byte[] automatically.
+   * Initialize the LazyObject with the parameters, wrapping the byte[]
+   * automatically.
    */
-  public static void initLazyObject(LazyObject lo, byte[] data, int start, int length) {
+  public static void initLazyObject(LazyObject lo, byte[] data, int start,
+      int length) {
     ByteArrayRef b = new ByteArrayRef();
     b.setData(data);
-    lo.init(b, start, length);    
+    lo.init(b, start, length);
   }
+
   /**
    * Test the LazyByte class.
    */
   public void testLazyByte() throws Throwable {
     try {
-      LazyByte b = new LazyByte(LazyPrimitiveObjectInspectorFactory.lazyByteObjectInspector);
-      initLazyObject(b,new byte[]{'0'}, 0, 0);
+      LazyByte b = new LazyByte(
+          LazyPrimitiveObjectInspectorFactory.lazyByteObjectInspector);
+      initLazyObject(b, new byte[] { '0' }, 0, 0);
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '0' }, 0, 1);
+      assertEquals(new ByteWritable((byte) 0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '0' }, 0, 2);
+      assertEquals(new ByteWritable((byte) 0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '0' }, 0, 2);
+      assertEquals(new ByteWritable((byte) 0), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 1);
+      assertEquals(new ByteWritable((byte) 1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '-', '1' }, 1, 2);
+      assertEquals(new ByteWritable((byte) -1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '+', '1' }, 1, 2);
+      assertEquals(new ByteWritable((byte) 1), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '1', '2', '8' }, 0, 4);
+      assertEquals(new ByteWritable((byte) -128), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '1', '2', '7' }, 0, 4);
+      assertEquals(new ByteWritable((byte) 127), b.getWritableObject());
+
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 2);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'0'}, 0, 1);
-      assertEquals(new ByteWritable((byte)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '0'}, 0, 2);
-      assertEquals(new ByteWritable((byte)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '0'}, 0, 2);
-      assertEquals(new ByteWritable((byte)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 1);
-      assertEquals(new ByteWritable((byte)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '-', '1'}, 1, 2);
-      assertEquals(new ByteWritable((byte)-1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '+', '1'}, 1, 2);
-      assertEquals(new ByteWritable((byte)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '8'}, 0, 4);
-      assertEquals(new ByteWritable((byte)-128), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '7'}, 0, 4);
-      assertEquals(new ByteWritable((byte)127), b.getWritableObject());
-      
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 2);
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '8'}, 0, 4);
+      initLazyObject(b, new byte[] { '+', '1', '2', '8' }, 0, 4);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '9'}, 0, 4);
+      initLazyObject(b, new byte[] { '-', '1', '2', '9' }, 0, 4);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '3'}, 0, 1);
+      initLazyObject(b, new byte[] { '-', '1', '2', '3' }, 0, 1);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '3'}, 0, 1);
+      initLazyObject(b, new byte[] { '+', '1', '2', '3' }, 0, 1);
       assertNull(b.getWritableObject());
-     
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
   }
-    
+
   /**
    * Test the LazyShort class.
    */
   public void testLazyShort() throws Throwable {
     try {
-      LazyShort b = new LazyShort(LazyPrimitiveObjectInspectorFactory.lazyShortObjectInspector);
-      initLazyObject(b,new byte[]{'0'}, 0, 0);
+      LazyShort b = new LazyShort(
+          LazyPrimitiveObjectInspectorFactory.lazyShortObjectInspector);
+      initLazyObject(b, new byte[] { '0' }, 0, 0);
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '0' }, 0, 1);
+      assertEquals(new ShortWritable((short) 0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '0' }, 0, 2);
+      assertEquals(new ShortWritable((short) 0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '0' }, 0, 2);
+      assertEquals(new ShortWritable((short) 0), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 1);
+      assertEquals(new ShortWritable((short) 1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '-', '1' }, 1, 2);
+      assertEquals(new ShortWritable((short) -1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '+', '1' }, 1, 2);
+      assertEquals(new ShortWritable((short) 1), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '1', '2', '8' }, 0, 4);
+      assertEquals(new ShortWritable((short) -128), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '1', '2', '7' }, 0, 4);
+      assertEquals(new ShortWritable((short) 127), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '3', '2', '7', '6', '8' }, 0, 6);
+      assertEquals(new ShortWritable((short) -32768), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '3', '2', '7', '6', '7' }, 0, 6);
+      assertEquals(new ShortWritable((short) 32767), b.getWritableObject());
+
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 2);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'0'}, 0, 1);
-      assertEquals(new ShortWritable((short)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '0'}, 0, 2);
-      assertEquals(new ShortWritable((short)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '0'}, 0, 2);
-      assertEquals(new ShortWritable((short)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 1);
-      assertEquals(new ShortWritable((short)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '-', '1'}, 1, 2);
-      assertEquals(new ShortWritable((short)-1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '+', '1'}, 1, 2);
-      assertEquals(new ShortWritable((short)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '8'}, 0, 4);
-      assertEquals(new ShortWritable((short)-128), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '7'}, 0, 4);
-      assertEquals(new ShortWritable((short)127), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '3', '2', '7', '6', '8'}, 0, 6);
-      assertEquals(new ShortWritable((short)-32768), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '3', '2', '7', '6', '7'}, 0, 6);
-      assertEquals(new ShortWritable((short)32767), b.getWritableObject());
-
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 2);
+      initLazyObject(b, new byte[] { '-', '3', '2', '7', '6', '9' }, 0, 6);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '3', '2', '7', '6', '9'}, 0, 6);
+      initLazyObject(b, new byte[] { '+', '3', '2', '7', '6', '8' }, 0, 6);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '3', '2', '7', '6', '8'}, 0, 6);
+      initLazyObject(b, new byte[] { '-', '1', '2', '3' }, 0, 1);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '3'}, 0, 1);
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '3'}, 0, 1);
+      initLazyObject(b, new byte[] { '+', '1', '2', '3' }, 0, 1);
       assertNull(b.getWritableObject());
 
     } catch (Throwable e) {
@@ -128,50 +130,54 @@
       throw e;
     }
   }
-     
 
   /**
    * Test the LazyInteger class.
    */
   public void testLazyInteger() throws Throwable {
     try {
-      LazyInteger b = new LazyInteger(LazyPrimitiveObjectInspectorFactory.lazyIntObjectInspector);
-      initLazyObject(b,new byte[]{'0'}, 0, 0);
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'0'}, 0, 1);
-      assertEquals(new IntWritable((int)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '0'}, 0, 2);
-      assertEquals(new IntWritable((int)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '0'}, 0, 2);
-      assertEquals(new IntWritable((int)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 1);
-      assertEquals(new IntWritable((int)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '-', '1'}, 1, 2);
-      assertEquals(new IntWritable((int)-1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '+', '1'}, 1, 2);
-      assertEquals(new IntWritable((int)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '8'}, 0, 4);
-      assertEquals(new IntWritable((int)-128), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '7'}, 0, 4);
-      assertEquals(new IntWritable((int)127), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '3', '2', '7', '6', '8'}, 0, 6);
-      assertEquals(new IntWritable((int)-32768), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '3', '2', '7', '6', '7'}, 0, 6);
-      assertEquals(new IntWritable((int)32767), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '2', '1', '4', '7', '4', '8', '3', '6', '4', '8'}, 0, 11);
-      assertEquals(new IntWritable((int)-2147483648), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '2', '1', '4', '7', '4', '8', '3', '6', '4', '7'}, 0, 11);
-      assertEquals(new IntWritable((int)2147483647), b.getWritableObject());
-
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 2);
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '2', '1', '4', '7', '4', '8', '3', '6', '4', '9'}, 0, 11);
+      LazyInteger b = new LazyInteger(
+          LazyPrimitiveObjectInspectorFactory.lazyIntObjectInspector);
+      initLazyObject(b, new byte[] { '0' }, 0, 0);
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '0' }, 0, 1);
+      assertEquals(new IntWritable(0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '0' }, 0, 2);
+      assertEquals(new IntWritable(0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '0' }, 0, 2);
+      assertEquals(new IntWritable(0), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 1);
+      assertEquals(new IntWritable(1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '-', '1' }, 1, 2);
+      assertEquals(new IntWritable(-1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '+', '1' }, 1, 2);
+      assertEquals(new IntWritable(1), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '1', '2', '8' }, 0, 4);
+      assertEquals(new IntWritable(-128), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '1', '2', '7' }, 0, 4);
+      assertEquals(new IntWritable(127), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '3', '2', '7', '6', '8' }, 0, 6);
+      assertEquals(new IntWritable(-32768), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '3', '2', '7', '6', '7' }, 0, 6);
+      assertEquals(new IntWritable(32767), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '8' }, 0, 11);
+      assertEquals(new IntWritable(-2147483648), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '7' }, 0, 11);
+      assertEquals(new IntWritable(2147483647), b.getWritableObject());
+
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 2);
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '9' }, 0, 11);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '2', '1', '4', '7', '4', '8', '3', '6', '4', '8'}, 0, 11);
+      initLazyObject(b, new byte[] { '+', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '8' }, 0, 11);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '3'}, 0, 1);
+      initLazyObject(b, new byte[] { '-', '1', '2', '3' }, 0, 1);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '3'}, 0, 1);
+      initLazyObject(b, new byte[] { '+', '1', '2', '3' }, 0, 1);
       assertNull(b.getWritableObject());
 
     } catch (Throwable e) {
@@ -179,58 +185,62 @@
       throw e;
     }
   }
-    
-  
+
   /**
    * Test the LazyLong class.
    */
   public void testLazyLong() throws Throwable {
     try {
-      LazyLong b = new LazyLong(LazyPrimitiveObjectInspectorFactory.lazyLongObjectInspector);
-      initLazyObject(b,new byte[]{'0'}, 0, 0);
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'0'}, 0, 1);
-      assertEquals(new LongWritable((long)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '0'}, 0, 2);
-      assertEquals(new LongWritable((long)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '0'}, 0, 2);
-      assertEquals(new LongWritable((long)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 1);
-      assertEquals(new LongWritable((long)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '-', '1'}, 1, 2);
-      assertEquals(new LongWritable((long)-1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '+', '1'}, 1, 2);
-      assertEquals(new LongWritable((long)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '8'}, 0, 4);
-      assertEquals(new LongWritable((long)-128), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '7'}, 0, 4);
-      assertEquals(new LongWritable((long)127), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '3', '2', '7', '6', '8'}, 0, 6);
-      assertEquals(new LongWritable((long)-32768), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '3', '2', '7', '6', '7'}, 0, 6);
-      assertEquals(new LongWritable((long)32767), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '2', '1', '4', '7', '4', '8', '3', '6', '4', '8'}, 0, 11);
-      assertEquals(new LongWritable((long)-2147483648), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '2', '1', '4', '7', '4', '8', '3', '6', '4', '7'}, 0, 11);
-      assertEquals(new LongWritable((long)2147483647), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '9', '2', '2', '3', '3', '7', '2', '0', '3', '6', '8', '5',
-          '4', '7', '7', '5', '8', '0', '8'}, 0, 20);
-      assertEquals(new LongWritable((long)-9223372036854775808L), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '9', '2', '2', '3', '3', '7', '2', '0', '3', '6', '8', '5',
-          '4', '7', '7', '5', '8', '0', '7'}, 0, 20);
-      assertEquals(new LongWritable((long)9223372036854775807L), b.getWritableObject());
-
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 2);
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '9', '2', '2', '3', '3', '7', '2', '0', '3', '6', '8', '5',
-          '4', '7', '7', '5', '8', '0', '9'}, 0, 20);
+      LazyLong b = new LazyLong(
+          LazyPrimitiveObjectInspectorFactory.lazyLongObjectInspector);
+      initLazyObject(b, new byte[] { '0' }, 0, 0);
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '0' }, 0, 1);
+      assertEquals(new LongWritable(0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '0' }, 0, 2);
+      assertEquals(new LongWritable(0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '0' }, 0, 2);
+      assertEquals(new LongWritable(0), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 1);
+      assertEquals(new LongWritable(1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '-', '1' }, 1, 2);
+      assertEquals(new LongWritable(-1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '+', '1' }, 1, 2);
+      assertEquals(new LongWritable(1), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '1', '2', '8' }, 0, 4);
+      assertEquals(new LongWritable(-128), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '1', '2', '7' }, 0, 4);
+      assertEquals(new LongWritable(127), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '3', '2', '7', '6', '8' }, 0, 6);
+      assertEquals(new LongWritable(-32768), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '3', '2', '7', '6', '7' }, 0, 6);
+      assertEquals(new LongWritable(32767), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '8' }, 0, 11);
+      assertEquals(new LongWritable(-2147483648), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '7' }, 0, 11);
+      assertEquals(new LongWritable(2147483647), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '9', '2', '2', '3', '3', '7', '2',
+          '0', '3', '6', '8', '5', '4', '7', '7', '5', '8', '0', '8' }, 0, 20);
+      assertEquals(new LongWritable(-9223372036854775808L), b
+          .getWritableObject());
+      initLazyObject(b, new byte[] { '+', '9', '2', '2', '3', '3', '7', '2',
+          '0', '3', '6', '8', '5', '4', '7', '7', '5', '8', '0', '7' }, 0, 20);
+      assertEquals(new LongWritable(9223372036854775807L), b
+          .getWritableObject());
+
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 2);
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '9', '2', '2', '3', '3', '7', '2',
+          '0', '3', '6', '8', '5', '4', '7', '7', '5', '8', '0', '9' }, 0, 20);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '9', '2', '2', '3', '3', '7', '2', '0', '3', '6', '8', '5',
-          '4', '7', '7', '5', '8', '0', '8'}, 0, 20);
+      initLazyObject(b, new byte[] { '+', '9', '2', '2', '3', '3', '7', '2',
+          '0', '3', '6', '8', '5', '4', '7', '7', '5', '8', '0', '8' }, 0, 20);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '3'}, 0, 1);
+      initLazyObject(b, new byte[] { '-', '1', '2', '3' }, 0, 1);
       assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '3'}, 0, 1);
+      initLazyObject(b, new byte[] { '+', '1', '2', '3' }, 0, 1);
       assertNull(b.getWritableObject());
 
     } catch (Throwable e) {
@@ -244,150 +254,163 @@
    */
   public void testLazyDouble() throws Throwable {
     try {
-      LazyDouble b = new LazyDouble(LazyPrimitiveObjectInspectorFactory.lazyDoubleObjectInspector);
-      initLazyObject(b,new byte[]{'0'}, 0, 0);
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'0'}, 0, 1);
-      assertEquals(new DoubleWritable((double)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '0'}, 0, 2);
-      assertEquals(new DoubleWritable((double)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '0'}, 0, 2);
-      assertEquals(new DoubleWritable((double)-0.0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 1);
-      assertEquals(new DoubleWritable((double)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '-', '1'}, 1, 2);
-      assertEquals(new DoubleWritable((double)-1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'a', '+', '1'}, 1, 2);
-      assertEquals(new DoubleWritable((double)1), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '8'}, 0, 4);
-      assertEquals(new DoubleWritable((double)-128), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '7'}, 0, 4);
-      assertEquals(new DoubleWritable((double)127), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '3', '2', '7', '6', '8'}, 0, 6);
-      assertEquals(new DoubleWritable((double)-32768), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '3', '2', '7', '6', '7'}, 0, 6);
-      assertEquals(new DoubleWritable((double)32767), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '2', '1', '4', '7', '4', '8', '3', '6', '4', '8'}, 0, 11);
-      assertEquals(new DoubleWritable((double)-2147483648), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '2', '1', '4', '7', '4', '8', '3', '6', '4', '7'}, 0, 11);
-      assertEquals(new DoubleWritable((double)2147483647), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '9', '2', '2', '3', '3', '7', '2', '0', '3', '6', '8', '5',
-          '4', '7', '7', '5', '8', '0', '8'}, 0, 20);
-      assertEquals(new DoubleWritable((double)-9223372036854775808L), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '9', '2', '2', '3', '3', '7', '2', '0', '3', '6', '8', '5',
-          '4', '7', '7', '5', '8', '0', '7'}, 0, 20);
-      assertEquals(new DoubleWritable((long)9223372036854775807L), b.getWritableObject());
-
-      initLazyObject(b,new byte[]{'-', '3', '.', '7', '6', '8'}, 0, 6);
-      assertEquals(new DoubleWritable((double)-3.768), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '3', '.', '7', '6', '7'}, 0, 6);
-      assertEquals(new DoubleWritable((double)3.767), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '2', '.', '4', '7', '4', '8', '3', '6', 'e', '8'}, 0, 11);
-      assertEquals(new DoubleWritable((double)-2.474836e8), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '2', '.', '4', '7', '4', '8', '3', 'E', '-', '7'}, 0, 11);
-      assertEquals(new DoubleWritable((double)2.47483E-7), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '.', '4', '7', '4', '8', '3', '6', 'e', '8'}, 0, 10);
-      assertEquals(new DoubleWritable((double)-.474836e8), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '.', '4', '7', '4', '8', '3', 'E', '-', '7'}, 0, 10);
-      assertEquals(new DoubleWritable((double).47483E-7), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '2', '1', '4', '7', '4', '8', '3', '6', '4', '.'}, 0, 11);
-      assertEquals(new DoubleWritable((double)-214748364.), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '2', '1', '4', '7', '4', '8', '3', '6', '4', '.'}, 0, 11);
-      assertEquals(new DoubleWritable((double)+214748364.), b.getWritableObject());
-
-      initLazyObject(b,new byte[]{'.', '0'}, 0, 2);
-      assertEquals(new DoubleWritable((double).0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'0', '.'}, 0, 2);
-      assertEquals(new DoubleWritable((double)0.), b.getWritableObject());
-      
-      initLazyObject(b,new byte[]{'a', '1', 'b'}, 1, 2);
-      assertNull(b.getWritableObject());
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'.', '1', '2', '3'}, 0, 1);
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', '2', '3'}, 0, 1);
-      assertNull(b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', '2', '3'}, 0, 1);
-      assertNull(b.getWritableObject());
-      
-      initLazyObject(b,new byte[]{'-', '1', 'e', '3', '3', '3', '3', '3', '3'}, 0, 9);
-      assertEquals(new DoubleWritable(Double.NEGATIVE_INFINITY), b.getWritableObject());
-      initLazyObject(b,new byte[]{'+', '1', 'e', '3', '3', '3', '3', '3', '3'}, 0, 9);
-      assertEquals(new DoubleWritable(Double.POSITIVE_INFINITY), b.getWritableObject());
-
-      initLazyObject(b,new byte[]{'+', '1', 'e', '-', '3', '3', '3', '3', '3'}, 0, 8);
-      assertEquals(new DoubleWritable((double)0), b.getWritableObject());
-      initLazyObject(b,new byte[]{'-', '1', 'e', '-', '3', '3', '3', '3', '3'}, 0, 8);
-      assertEquals(new DoubleWritable((double)-0.0), b.getWritableObject());
-      
+      LazyDouble b = new LazyDouble(
+          LazyPrimitiveObjectInspectorFactory.lazyDoubleObjectInspector);
+      initLazyObject(b, new byte[] { '0' }, 0, 0);
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '0' }, 0, 1);
+      assertEquals(new DoubleWritable(0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '0' }, 0, 2);
+      assertEquals(new DoubleWritable(0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '0' }, 0, 2);
+      assertEquals(new DoubleWritable(-0.0), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 1);
+      assertEquals(new DoubleWritable(1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '-', '1' }, 1, 2);
+      assertEquals(new DoubleWritable(-1), b.getWritableObject());
+      initLazyObject(b, new byte[] { 'a', '+', '1' }, 1, 2);
+      assertEquals(new DoubleWritable(1), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '1', '2', '8' }, 0, 4);
+      assertEquals(new DoubleWritable(-128), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '1', '2', '7' }, 0, 4);
+      assertEquals(new DoubleWritable(127), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '3', '2', '7', '6', '8' }, 0, 6);
+      assertEquals(new DoubleWritable(-32768), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '3', '2', '7', '6', '7' }, 0, 6);
+      assertEquals(new DoubleWritable(32767), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '8' }, 0, 11);
+      assertEquals(new DoubleWritable(-2147483648), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '7' }, 0, 11);
+      assertEquals(new DoubleWritable(2147483647), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '9', '2', '2', '3', '3', '7', '2',
+          '0', '3', '6', '8', '5', '4', '7', '7', '5', '8', '0', '8' }, 0, 20);
+      assertEquals(new DoubleWritable(-9223372036854775808L), b
+          .getWritableObject());
+      initLazyObject(b, new byte[] { '+', '9', '2', '2', '3', '3', '7', '2',
+          '0', '3', '6', '8', '5', '4', '7', '7', '5', '8', '0', '7' }, 0, 20);
+      assertEquals(new DoubleWritable(9223372036854775807L), b
+          .getWritableObject());
+
+      initLazyObject(b, new byte[] { '-', '3', '.', '7', '6', '8' }, 0, 6);
+      assertEquals(new DoubleWritable(-3.768), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '3', '.', '7', '6', '7' }, 0, 6);
+      assertEquals(new DoubleWritable(3.767), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '2', '.', '4', '7', '4', '8', '3',
+          '6', 'e', '8' }, 0, 11);
+      assertEquals(new DoubleWritable(-2.474836e8), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '2', '.', '4', '7', '4', '8', '3',
+          'E', '-', '7' }, 0, 11);
+      assertEquals(new DoubleWritable(2.47483E-7), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '.', '4', '7', '4', '8', '3', '6',
+          'e', '8' }, 0, 10);
+      assertEquals(new DoubleWritable(-.474836e8), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '.', '4', '7', '4', '8', '3', 'E',
+          '-', '7' }, 0, 10);
+      assertEquals(new DoubleWritable(.47483E-7), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '.' }, 0, 11);
+      assertEquals(new DoubleWritable(-214748364.), b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '2', '1', '4', '7', '4', '8', '3',
+          '6', '4', '.' }, 0, 11);
+      assertEquals(new DoubleWritable(+214748364.), b.getWritableObject());
+
+      initLazyObject(b, new byte[] { '.', '0' }, 0, 2);
+      assertEquals(new DoubleWritable(.0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '0', '.' }, 0, 2);
+      assertEquals(new DoubleWritable(0.), b.getWritableObject());
+
+      initLazyObject(b, new byte[] { 'a', '1', 'b' }, 1, 2);
+      assertNull(b.getWritableObject());
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '.', '1', '2', '3' }, 0, 1);
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '1', '2', '3' }, 0, 1);
+      assertNull(b.getWritableObject());
+      initLazyObject(b, new byte[] { '+', '1', '2', '3' }, 0, 1);
+      assertNull(b.getWritableObject());
+
+      initLazyObject(b, new byte[] { '-', '1', 'e', '3', '3', '3', '3', '3',
+          '3' }, 0, 9);
+      assertEquals(new DoubleWritable(Double.NEGATIVE_INFINITY), b
+          .getWritableObject());
+      initLazyObject(b, new byte[] { '+', '1', 'e', '3', '3', '3', '3', '3',
+          '3' }, 0, 9);
+      assertEquals(new DoubleWritable(Double.POSITIVE_INFINITY), b
+          .getWritableObject());
+
+      initLazyObject(b, new byte[] { '+', '1', 'e', '-', '3', '3', '3', '3',
+          '3' }, 0, 8);
+      assertEquals(new DoubleWritable(0), b.getWritableObject());
+      initLazyObject(b, new byte[] { '-', '1', 'e', '-', '3', '3', '3', '3',
+          '3' }, 0, 8);
+      assertEquals(new DoubleWritable(-0.0), b.getWritableObject());
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
   }
-  
+
   /**
    * Test the LazyString class.
    */
   public void testLazyString() throws Throwable {
     try {
       LazyString b = new LazyString(LazyPrimitiveObjectInspectorFactory
-          .getLazyStringObjectInspector(false,(byte)0));
-      initLazyObject(b,new byte[]{'0'}, 0, 0);
+          .getLazyStringObjectInspector(false, (byte) 0));
+      initLazyObject(b, new byte[] { '0' }, 0, 0);
       assertEquals(new Text(""), b.getWritableObject());
-      initLazyObject(b,new byte[]{'0'}, 0, 1);
+      initLazyObject(b, new byte[] { '0' }, 0, 1);
       assertEquals(new Text("0"), b.getWritableObject());
-      initLazyObject(b,new byte[]{'0', '1', '2'}, 1, 1);
+      initLazyObject(b, new byte[] { '0', '1', '2' }, 1, 1);
       assertEquals(new Text("1"), b.getWritableObject());
-      
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
   }
-  
+
   public void testLazyIntegerWrite() throws Throwable {
     try {
       ByteStream.Output out = new ByteStream.Output();
-      
-      int[] tests = {0, -1, 1, -10, 10, -123, 123, 
-          Integer.MIN_VALUE, Integer.MIN_VALUE + 1,
-          Integer.MAX_VALUE, Integer.MAX_VALUE - 1};
-      for (int i=0; i<tests.length; i++) {
-        int v = tests[i];
+
+      int[] tests = { 0, -1, 1, -10, 10, -123, 123, Integer.MIN_VALUE,
+          Integer.MIN_VALUE + 1, Integer.MAX_VALUE, Integer.MAX_VALUE - 1 };
+      for (int v : tests) {
         out.reset();
         LazyInteger.writeUTF8(out, v);
         Text t = new Text();
         t.set(out.getData(), 0, out.getCount());
         assertEquals(String.valueOf(v), t.toString());
       }
-      
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
-  }  
+  }
 
   public void testLazyLongWrite() throws Throwable {
     try {
       ByteStream.Output out = new ByteStream.Output();
-      
-      long[] tests = {0L, -1, 1, -10, 10, -123, 123,
-          Long.MIN_VALUE, Long.MIN_VALUE + 1, 
-          Long.MAX_VALUE, Long.MAX_VALUE - 1};
-      for (int i=0; i<tests.length; i++) {
-        long v = tests[i];
+
+      long[] tests = { 0L, -1, 1, -10, 10, -123, 123, Long.MIN_VALUE,
+          Long.MIN_VALUE + 1, Long.MAX_VALUE, Long.MAX_VALUE - 1 };
+      for (long v : tests) {
         out.reset();
         LazyLong.writeUTF8(out, v);
         Text t = new Text();
         t.set(out.getData(), 0, out.getCount());
         assertEquals(String.valueOf(v), t.toString());
       }
-      
+
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;
     }
-  }  
+  }
 
 }