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:17:28 UTC

svn commit: r901608 [2/2] - in /hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary: ./ objectinspector/

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java?rev=901608&r1=901607&r2=901608&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java Thu Jan 21 09:16:48 2010
@@ -23,8 +23,8 @@
 
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.lazybinary.objectinspector.LazyBinaryObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
@@ -36,206 +36,230 @@
 import org.apache.hadoop.io.WritableUtils;
 
 public class LazyBinaryUtils {
- 
+
   /**
-   * Convert the byte array to an int starting from the given offset.
-   * Refer to code by aeden on DZone Snippets: 
-   * @param b       the byte array
-   * @param offset  the array offset
-   * @return        the integer
+   * Convert the byte array to an int starting from the given offset. Refer to
+   * code by aeden on DZone Snippets:
+   * 
+   * @param b
+   *          the byte array
+   * @param offset
+   *          the array offset
+   * @return the integer
    */
   public static int byteArrayToInt(byte[] b, int offset) {
-      int value = 0;
-      for (int i = 0; i < 4; i++) {
-          int shift = (4 - 1 - i) * 8;
-          value += (b[i + offset] & 0x000000FF) << shift;
-      }
-      return value;
+    int value = 0;
+    for (int i = 0; i < 4; i++) {
+      int shift = (4 - 1 - i) * 8;
+      value += (b[i + offset] & 0x000000FF) << shift;
+    }
+    return value;
   }
-    
+
   /**
    * Convert the byte array to a long starting from the given offset.
-   * @param b       the byte array
-   * @param offset  the array offset
-   * @return        the long
+   * 
+   * @param b
+   *          the byte array
+   * @param offset
+   *          the array offset
+   * @return the long
    */
   public static long byteArrayToLong(byte[] b, int offset) {
-      long value = 0;
-      for (int i = 0; i < 8; i++) {
-          int shift = (8 - 1 - i) * 8;
-          value += ((long) (b[i + offset] & 0x00000000000000FF)) << shift;
-      }
-      return value;
+    long value = 0;
+    for (int i = 0; i < 8; i++) {
+      int shift = (8 - 1 - i) * 8;
+      value += ((long) (b[i + offset] & 0x00000000000000FF)) << shift;
+    }
+    return value;
   }
 
   /**
    * Convert the byte array to a short starting from the given offset.
-   * @param b       the byte array
-   * @param offset  the array offset
-   * @return        the short
+   * 
+   * @param b
+   *          the byte array
+   * @param offset
+   *          the array offset
+   * @return the short
    */
   public static short byteArrayToShort(byte[] b, int offset) {
-      short value = 0;
-      value += (b[offset  ] & 0x000000FF) << 8;
-      value += (b[offset+1] & 0x000000FF);
-      return value;
+    short value = 0;
+    value += (b[offset] & 0x000000FF) << 8;
+    value += (b[offset + 1] & 0x000000FF);
+    return value;
   }
-  
+
   /**
-   * Record is the unit that data is serialized in.
-   * A record includes two parts. The first part stores the
-   * size of the element and the second part stores the 
-   * real element.
-   *            size         element
-   * record -> |----|-------------------------|
-   *
-   * A RecordInfo stores two information of a record, 
-   * the size of the "size" part which is the element offset
-   * and the size of the element part which is element size. 
-   */  
+   * Record is the unit that data is serialized in. A record includes two parts.
+   * The first part stores the size of the element and the second part stores
+   * the real element. size element record -> |----|-------------------------|
+   * 
+   * A RecordInfo stores two information of a record, the size of the "size"
+   * part which is the element offset and the size of the element part which is
+   * element size.
+   */
   public static class RecordInfo {
-    public RecordInfo () {
+    public RecordInfo() {
       elementOffset = 0;
-      elementSize   = 0;
+      elementSize = 0;
     }
+
     public byte elementOffset;
-    public int  elementSize;    
+    public int elementSize;
   }
 
   static VInt vInt = new LazyBinaryUtils.VInt();
+
   /**
-   * Check a particular field and set its size and offset in bytes 
-   * based on the field type and the bytes arrays.
-   *  
-   * For void, boolean, byte, short, int, long, float and double, 
-   * there is no offset and the size is fixed. For string, map, 
-   * list, struct, the first four bytes are used to store the size.
-   * So the offset is 4 and the size is computed by concating the 
-   * first four bytes together. The first four bytes are defined 
-   * with respect to the offset in the bytes arrays. 
-   * 
-   * @param objectInspector   object inspector of the field
-   * @param bytes     bytes arrays store the table row
-   * @param offset    offset of this field
-   * @param recordInfo  modify this byteinfo object and return it
-   */
-  public static void checkObjectByteInfo(ObjectInspector objectInspector, byte[] bytes, int offset, RecordInfo recordInfo) {
-    Category category = objectInspector.getCategory(); 
+   * Check a particular field and set its size and offset in bytes based on the
+   * field type and the bytes arrays.
+   * 
+   * For void, boolean, byte, short, int, long, float and double, there is no
+   * offset and the size is fixed. For string, map, list, struct, the first four
+   * bytes are used to store the size. So the offset is 4 and the size is
+   * computed by concating the first four bytes together. The first four bytes
+   * are defined with respect to the offset in the bytes arrays.
+   * 
+   * @param objectInspector
+   *          object inspector of the field
+   * @param bytes
+   *          bytes arrays store the table row
+   * @param offset
+   *          offset of this field
+   * @param recordInfo
+   *          modify this byteinfo object and return it
+   */
+  public static void checkObjectByteInfo(ObjectInspector objectInspector,
+      byte[] bytes, int offset, RecordInfo recordInfo) {
+    Category category = objectInspector.getCategory();
     switch (category) {
     case PRIMITIVE:
-      PrimitiveCategory primitiveCategory = ((PrimitiveObjectInspector)objectInspector).getPrimitiveCategory(); 
+      PrimitiveCategory primitiveCategory = ((PrimitiveObjectInspector) objectInspector)
+          .getPrimitiveCategory();
       switch (primitiveCategory) {
       case VOID:
       case BOOLEAN:
       case BYTE:
         recordInfo.elementOffset = 0;
-        recordInfo.elementSize   = 1;
+        recordInfo.elementSize = 1;
         break;
       case SHORT:
         recordInfo.elementOffset = 0;
-        recordInfo.elementSize   = 2;
+        recordInfo.elementSize = 2;
         break;
       case FLOAT:
         recordInfo.elementOffset = 0;
-        recordInfo.elementSize   = 4;
-        break; 
+        recordInfo.elementSize = 4;
+        break;
       case DOUBLE:
         recordInfo.elementOffset = 0;
-        recordInfo.elementSize   = 8;
-        break; 
+        recordInfo.elementSize = 8;
+        break;
       case INT:
         recordInfo.elementOffset = 0;
-        recordInfo.elementSize   = WritableUtils.decodeVIntSize(bytes[offset]);
+        recordInfo.elementSize = WritableUtils.decodeVIntSize(bytes[offset]);
         break;
       case LONG:
         recordInfo.elementOffset = 0;
-        recordInfo.elementSize   = WritableUtils.decodeVIntSize(bytes[offset]);
+        recordInfo.elementSize = WritableUtils.decodeVIntSize(bytes[offset]);
         break;
       case STRING:
         // using vint instead of 4 bytes
         LazyBinaryUtils.readVInt(bytes, offset, vInt);
         recordInfo.elementOffset = vInt.length;
-        recordInfo.elementSize   = vInt.value;
+        recordInfo.elementSize = vInt.value;
         break;
-        default: {
-          throw new RuntimeException("Unrecognized primitive type: " + primitiveCategory);
-        }
+      default: {
+        throw new RuntimeException("Unrecognized primitive type: "
+            + primitiveCategory);
+      }
       }
       break;
     case LIST:
     case MAP:
     case STRUCT:
       recordInfo.elementOffset = 4;
-      recordInfo.elementSize   = LazyBinaryUtils.byteArrayToInt(bytes, offset);
+      recordInfo.elementSize = LazyBinaryUtils.byteArrayToInt(bytes, offset);
       break;
-      default : {      
-        throw new RuntimeException("Unrecognized non-primitive type: " + category);
-      }
+    default: {
+      throw new RuntimeException("Unrecognized non-primitive type: " + category);
+    }
     }
   }
-  
+
   /**
-   *  A zero-compressed encoded long
+   * A zero-compressed encoded long
    */
   public static class VLong {
     public VLong() {
-      value  = 0;
+      value = 0;
       length = 0;
     }
+
     public long value;
-    public byte length;  
+    public byte length;
   };
-  
+
   /**
    * Reads a zero-compressed encoded long from a byte array and returns it.
-   * @param bytes   the byte array
-   * @param offset  offset of the array to read from
-   * @param vlong   storing the deserialized long and its size in byte
+   * 
+   * @param bytes
+   *          the byte array
+   * @param offset
+   *          offset of the array to read from
+   * @param vlong
+   *          storing the deserialized long and its size in byte
    */
   public static void readVLong(byte[] bytes, int offset, VLong vlong) {
     byte firstByte = bytes[offset];
-    vlong.length = (byte)WritableUtils.decodeVIntSize(firstByte);
+    vlong.length = (byte) WritableUtils.decodeVIntSize(firstByte);
     if (vlong.length == 1) {
       vlong.value = firstByte;
       return;
     }
     long i = 0;
-    for (int idx = 0; idx < vlong.length-1; idx++) {
-      byte b = bytes[offset+1+idx];
+    for (int idx = 0; idx < vlong.length - 1; idx++) {
+      byte b = bytes[offset + 1 + idx];
       i = i << 8;
       i = i | (b & 0xFF);
     }
     vlong.value = (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
   }
-  
+
   /**
-   *  A zero-compressed encoded integer
+   * A zero-compressed encoded integer
    */
   public static class VInt {
     public VInt() {
-      value  = 0;
+      value = 0;
       length = 0;
     }
-    public int  value;
-    public byte length;  
+
+    public int value;
+    public byte length;
   };
 
   /**
    * Reads a zero-compressed encoded int from a byte array and returns it.
-   * @param bytes   the byte array
-   * @param offset  offset of the array to read from
-   * @param vInt    storing the deserialized int and its size in byte
+   * 
+   * @param bytes
+   *          the byte array
+   * @param offset
+   *          offset of the array to read from
+   * @param vInt
+   *          storing the deserialized int and its size in byte
    */
   public static void readVInt(byte[] bytes, int offset, VInt vInt) {
     byte firstByte = bytes[offset];
-    vInt.length = (byte)WritableUtils.decodeVIntSize(firstByte);
+    vInt.length = (byte) WritableUtils.decodeVIntSize(firstByte);
     if (vInt.length == 1) {
       vInt.value = firstByte;
       return;
     }
     int i = 0;
-    for (int idx = 0; idx < vInt.length-1; idx++) {
-      byte b = bytes[offset+1+idx];
+    for (int idx = 0; idx < vInt.length - 1; idx++) {
+      byte b = bytes[offset + 1 + idx];
       i = i << 8;
       i = i | (b & 0xFF);
     }
@@ -244,91 +268,110 @@
 
   /**
    * Writes a zero-compressed encoded int to a byte array.
-   * @param byteStream   the byte array/stream
-   * @param i            the int
+   * 
+   * @param byteStream
+   *          the byte array/stream
+   * @param i
+   *          the int
    */
   public static void writeVInt(Output byteStream, int i) {
     writeVLong(byteStream, i);
   }
-  
+
   /**
    * Write a zero-compressed encoded long to a byte array.
-   * @param byteStream   the byte array/stream
-   * @param l            the long
+   * 
+   * @param byteStream
+   *          the byte array/stream
+   * @param l
+   *          the long
    */
   public static void writeVLong(Output byteStream, long l) {
     if (l >= -112 && l <= 127) {
-      byteStream.write((byte)l);
+      byteStream.write((byte) l);
       return;
     }
-      
+
     int len = -112;
     if (l < 0) {
       l ^= -1L; // take one's complement'
       len = -120;
     }
-      
+
     long tmp = l;
     while (tmp != 0) {
       tmp = tmp >> 8;
       len--;
     }
-      
-    byteStream.write((byte)len);
-      
+
+    byteStream.write((byte) len);
+
     len = (len < -120) ? -(len + 120) : -(len + 112);
-      
+
     for (int idx = len; idx != 0; idx--) {
       int shiftbits = (idx - 1) * 8;
       long mask = 0xFFL << shiftbits;
-      byteStream.write((byte)((l & mask) >> shiftbits));
+      byteStream.write((byte) ((l & mask) >> shiftbits));
     }
-  }  
-  
+  }
+
   static HashMap<TypeInfo, ObjectInspector> cachedLazyBinaryObjectInspector = new HashMap<TypeInfo, ObjectInspector>();
 
   /**
-   * Returns the lazy binary object inspector that can be used to inspect an 
+   * Returns the lazy binary object inspector that can be used to inspect an
    * lazy binary object of that typeInfo
    * 
-   * For primitive types, we use the standard writable object inspector.  
+   * For primitive types, we use the standard writable object inspector.
    */
-  public static ObjectInspector getLazyBinaryObjectInspectorFromTypeInfo(TypeInfo typeInfo) {
+  public static ObjectInspector getLazyBinaryObjectInspectorFromTypeInfo(
+      TypeInfo typeInfo) {
     ObjectInspector result = cachedLazyBinaryObjectInspector.get(typeInfo);
     if (result == null) {
-      switch(typeInfo.getCategory()) {
-        case PRIMITIVE: {
-          result = PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(
-              ((PrimitiveTypeInfo)typeInfo).getPrimitiveCategory());
-          break;
-        }
-        case LIST: {
-          ObjectInspector elementObjectInspector = getLazyBinaryObjectInspectorFromTypeInfo(
-              ((ListTypeInfo)typeInfo).getListElementTypeInfo());
-          result = LazyBinaryObjectInspectorFactory.getLazyBinaryListObjectInspector(elementObjectInspector);
-          break;
-        }
-        case MAP: {
-          MapTypeInfo mapTypeInfo = (MapTypeInfo)typeInfo;
-          ObjectInspector keyObjectInspector = getLazyBinaryObjectInspectorFromTypeInfo(mapTypeInfo.getMapKeyTypeInfo());
-          ObjectInspector valueObjectInspector = getLazyBinaryObjectInspectorFromTypeInfo(mapTypeInfo.getMapValueTypeInfo());
-          result = LazyBinaryObjectInspectorFactory.getLazyBinaryMapObjectInspector(keyObjectInspector, valueObjectInspector);
-          break;
-        }
-        case STRUCT: {
-          StructTypeInfo structTypeInfo = (StructTypeInfo)typeInfo;
-          List<String> fieldNames = structTypeInfo.getAllStructFieldNames();
-          List<TypeInfo> fieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos();
-          List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(fieldTypeInfos.size());
-          for(int i=0; i<fieldTypeInfos.size(); i++) {
-            fieldObjectInspectors.add(getLazyBinaryObjectInspectorFromTypeInfo(fieldTypeInfos.get(i)));
-          }
-          result = LazyBinaryObjectInspectorFactory.getLazyBinaryStructObjectInspector(fieldNames, fieldObjectInspectors);
-          break;
-        }
-        default: {
-          result = null;
+      switch (typeInfo.getCategory()) {
+      case PRIMITIVE: {
+        result = PrimitiveObjectInspectorFactory
+            .getPrimitiveWritableObjectInspector(((PrimitiveTypeInfo) typeInfo)
+                .getPrimitiveCategory());
+        break;
+      }
+      case LIST: {
+        ObjectInspector elementObjectInspector = getLazyBinaryObjectInspectorFromTypeInfo(((ListTypeInfo) typeInfo)
+            .getListElementTypeInfo());
+        result = LazyBinaryObjectInspectorFactory
+            .getLazyBinaryListObjectInspector(elementObjectInspector);
+        break;
+      }
+      case MAP: {
+        MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo;
+        ObjectInspector keyObjectInspector = getLazyBinaryObjectInspectorFromTypeInfo(mapTypeInfo
+            .getMapKeyTypeInfo());
+        ObjectInspector valueObjectInspector = getLazyBinaryObjectInspectorFromTypeInfo(mapTypeInfo
+            .getMapValueTypeInfo());
+        result = LazyBinaryObjectInspectorFactory
+            .getLazyBinaryMapObjectInspector(keyObjectInspector,
+                valueObjectInspector);
+        break;
+      }
+      case STRUCT: {
+        StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo;
+        List<String> fieldNames = structTypeInfo.getAllStructFieldNames();
+        List<TypeInfo> fieldTypeInfos = structTypeInfo
+            .getAllStructFieldTypeInfos();
+        List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(
+            fieldTypeInfos.size());
+        for (int i = 0; i < fieldTypeInfos.size(); i++) {
+          fieldObjectInspectors
+              .add(getLazyBinaryObjectInspectorFromTypeInfo(fieldTypeInfos
+                  .get(i)));
         }
+        result = LazyBinaryObjectInspectorFactory
+            .getLazyBinaryStructObjectInspector(fieldNames,
+                fieldObjectInspectors);
+        break;
+      }
+      default: {
+        result = null;
+      }
       }
       cachedLazyBinaryObjectInspector.put(typeInfo, result);
     }

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryListObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryListObjectInspector.java?rev=901608&r1=901607&r2=901608&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryListObjectInspector.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryListObjectInspector.java Thu Jan 21 09:16:48 2010
@@ -20,8 +20,8 @@
 import java.util.List;
 
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryArray;
-import org.apache.hadoop.hive.serde2.objectinspector.StandardListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StandardListObjectInspector;
 
 /**
  * ObjectInspector for LazyBinaryList

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryMapObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryMapObjectInspector.java?rev=901608&r1=901607&r2=901608&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryMapObjectInspector.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryMapObjectInspector.java Thu Jan 21 09:16:48 2010
@@ -25,6 +25,7 @@
 
 /**
  * ObjectInspector for LazyBinaryMap
+ * 
  * @see LazyBinaryMap
  */
 public class LazyBinaryMapObjectInspector extends StandardMapObjectInspector {
@@ -39,7 +40,7 @@
     if (data == null) {
       return null;
     }
-    return ((LazyBinaryMap)data).getMap();
+    return ((LazyBinaryMap) data).getMap();
   }
 
   @Override
@@ -47,7 +48,7 @@
     if (data == null) {
       return -1;
     }
-    return ((LazyBinaryMap)data).getMapSize();  
+    return ((LazyBinaryMap) data).getMapSize();
   }
 
   @Override
@@ -55,6 +56,6 @@
     if (data == null) {
       return -1;
     }
-    return ((LazyBinaryMap)data).getMapValueElement(key);  
+    return ((LazyBinaryMap) data).getMapValueElement(key);
   }
 }

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryObjectInspectorFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryObjectInspectorFactory.java?rev=901608&r1=901607&r2=901608&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryObjectInspectorFactory.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryObjectInspectorFactory.java Thu Jan 21 09:16:48 2010
@@ -7,55 +7,62 @@
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 /**
- * ObjectInspectorFactory is the primary way to create new ObjectInspector instances.
+ * ObjectInspectorFactory is the primary way to create new ObjectInspector
+ * instances.
  * 
- * SerDe classes should call the static functions in this library to create an ObjectInspector
- * to return to the caller of SerDe2.getObjectInspector().
+ * SerDe classes should call the static functions in this library to create an
+ * ObjectInspector to return to the caller of SerDe2.getObjectInspector().
  * 
- * The reason of having caches here is that ObjectInspectors do not have an internal 
- * state - so ObjectInspectors with the same construction parameters should
- * result in exactly the same ObjectInspector.
+ * The reason of having caches here is that ObjectInspectors do not have an
+ * internal state - so ObjectInspectors with the same construction parameters
+ * should result in exactly the same ObjectInspector.
  */
 
 public class LazyBinaryObjectInspectorFactory {
 
-  static HashMap<ArrayList<Object>, LazyBinaryStructObjectInspector> cachedLazyBinaryStructObjectInspector =
-    new HashMap<ArrayList<Object>, LazyBinaryStructObjectInspector>(); 
-  public static LazyBinaryStructObjectInspector getLazyBinaryStructObjectInspector(List<String> structFieldNames, 
+  static HashMap<ArrayList<Object>, LazyBinaryStructObjectInspector> cachedLazyBinaryStructObjectInspector = new HashMap<ArrayList<Object>, LazyBinaryStructObjectInspector>();
+
+  public static LazyBinaryStructObjectInspector getLazyBinaryStructObjectInspector(
+      List<String> structFieldNames,
       List<ObjectInspector> structFieldObjectInspectors) {
     ArrayList<Object> signature = new ArrayList<Object>();
     signature.add(structFieldNames);
     signature.add(structFieldObjectInspectors);
-    LazyBinaryStructObjectInspector result = cachedLazyBinaryStructObjectInspector.get(signature);
+    LazyBinaryStructObjectInspector result = cachedLazyBinaryStructObjectInspector
+        .get(signature);
     if (result == null) {
-      result = new LazyBinaryStructObjectInspector(structFieldNames, structFieldObjectInspectors);
+      result = new LazyBinaryStructObjectInspector(structFieldNames,
+          structFieldObjectInspectors);
       cachedLazyBinaryStructObjectInspector.put(signature, result);
     }
     return result;
   }
-  
-  static HashMap<ArrayList<Object>, LazyBinaryListObjectInspector> cachedLazyBinaryListObjectInspector =
-    new HashMap<ArrayList<Object>, LazyBinaryListObjectInspector>(); 
-  public static LazyBinaryListObjectInspector getLazyBinaryListObjectInspector( 
+
+  static HashMap<ArrayList<Object>, LazyBinaryListObjectInspector> cachedLazyBinaryListObjectInspector = new HashMap<ArrayList<Object>, LazyBinaryListObjectInspector>();
+
+  public static LazyBinaryListObjectInspector getLazyBinaryListObjectInspector(
       ObjectInspector listElementObjectInspector) {
     ArrayList<Object> signature = new ArrayList<Object>();
     signature.add(listElementObjectInspector);
-    LazyBinaryListObjectInspector result = cachedLazyBinaryListObjectInspector.get(signature);
+    LazyBinaryListObjectInspector result = cachedLazyBinaryListObjectInspector
+        .get(signature);
     if (result == null) {
       result = new LazyBinaryListObjectInspector(listElementObjectInspector);
       cachedLazyBinaryListObjectInspector.put(signature, result);
     }
     return result;
   }
-  
-  static HashMap<ArrayList<Object>, LazyBinaryMapObjectInspector> cachedLazyBinaryMapObjectInspector =
-    new HashMap<ArrayList<Object>, LazyBinaryMapObjectInspector>(); 
-  public static LazyBinaryMapObjectInspector getLazyBinaryMapObjectInspector( 
-      ObjectInspector mapKeyObjectInspector, ObjectInspector mapValueObjectInspector) {
+
+  static HashMap<ArrayList<Object>, LazyBinaryMapObjectInspector> cachedLazyBinaryMapObjectInspector = new HashMap<ArrayList<Object>, LazyBinaryMapObjectInspector>();
+
+  public static LazyBinaryMapObjectInspector getLazyBinaryMapObjectInspector(
+      ObjectInspector mapKeyObjectInspector,
+      ObjectInspector mapValueObjectInspector) {
     ArrayList<Object> signature = new ArrayList<Object>();
     signature.add(mapKeyObjectInspector);
     signature.add(mapValueObjectInspector);
-    LazyBinaryMapObjectInspector result = cachedLazyBinaryMapObjectInspector.get(signature);
+    LazyBinaryMapObjectInspector result = cachedLazyBinaryMapObjectInspector
+        .get(signature);
     if (result == null) {
       result = new LazyBinaryMapObjectInspector(mapKeyObjectInspector,
           mapValueObjectInspector);

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryStructObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryStructObjectInspector.java?rev=901608&r1=901607&r2=901608&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryStructObjectInspector.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryStructObjectInspector.java Thu Jan 21 09:16:48 2010
@@ -26,29 +26,32 @@
 
 /**
  * ObjectInspector for LazyBinaryStruct
+ * 
  * @see LazyBinaryStruct
  */
-public class LazyBinaryStructObjectInspector extends StandardStructObjectInspector {
+public class LazyBinaryStructObjectInspector extends
+    StandardStructObjectInspector {
 
-  protected LazyBinaryStructObjectInspector(List<String> structFieldNames, List<ObjectInspector> structFieldObjectInspectors) {
+  protected LazyBinaryStructObjectInspector(List<String> structFieldNames,
+      List<ObjectInspector> structFieldObjectInspectors) {
     super(structFieldNames, structFieldObjectInspectors);
   }
-  
+
   protected LazyBinaryStructObjectInspector(List<StructField> fields) {
-    super(fields);    
+    super(fields);
   }
-  
+
   @Override
   public Object getStructFieldData(Object data, StructField fieldRef) {
     if (data == null) {
       return null;
     }
-    LazyBinaryStruct struct = (LazyBinaryStruct)data;
+    LazyBinaryStruct struct = (LazyBinaryStruct) data;
     MyField f = (MyField) fieldRef;
-    
+
     int fieldID = f.getFieldID();
-    assert(fieldID >= 0 && fieldID < fields.size());
-    
+    assert (fieldID >= 0 && fieldID < fields.size());
+
     return struct.getField(fieldID);
   }
 
@@ -57,7 +60,7 @@
     if (data == null) {
       return null;
     }
-    LazyBinaryStruct struct = (LazyBinaryStruct)data;
+    LazyBinaryStruct struct = (LazyBinaryStruct) data;
     return struct.getFieldsAsList();
   }
 }