You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by dh...@apache.org on 2008/10/21 20:11:18 UTC

svn commit: r706704 [22/23] - in /hadoop/core/trunk: ./ src/contrib/hive/ src/contrib/hive/bin/ src/contrib/hive/cli/src/java/org/apache/hadoop/hive/cli/ src/contrib/hive/common/src/java/org/apache/hadoop/hive/conf/ src/contrib/hive/conf/ src/contrib/h...

Modified: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/MetadataTypedColumnsetSerDe.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/MetadataTypedColumnsetSerDe.java?rev=706704&r1=706703&r2=706704&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/MetadataTypedColumnsetSerDe.java (original)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/MetadataTypedColumnsetSerDe.java Tue Oct 21 11:11:05 2008
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.serde2;
 
-import java.io.UnsupportedEncodingException;
 import java.nio.charset.CharacterCodingException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -44,33 +43,24 @@
 
   public static final Log LOG = LogFactory.getLog(MetadataTypedColumnsetSerDe.class.getName());
 
-  public String getShortName() {
-    return shortName();
-  }
-
-
-  public static String shortName() {
-    return "simple_meta";
-  }
-
   static {
     StackTraceElement[] sTrace = new Exception().getStackTrace();
     String className = sTrace[0].getClassName();
     try {
-      SerDeUtils.registerSerDe(shortName(), Class.forName(className));
-      // For backward compatibility: this class replaces the following class.
-      SerDeUtils.registerSerDe("org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe", 
+      // For backward compatibility: this class replaces the columnsetSerDe class.
+      SerDeUtils.registerSerDe("org.apache.hadoop.hive.serde.thrift.columnsetSerDe", 
           Class.forName(className));
-    } catch(Exception e) {
+      } catch(Exception e) {
       throw new RuntimeException(e);
     }
   }
   
   final public static String DefaultSeparator = "\001";
-
   private String separator;
-  // constant for now, will make it configurable later.
-  private String nullString = "\\N"; 
+
+  final public static String defaultNullString = "\\N";
+  private String nullString; 
+
   private List<String> columnNames;
   private ObjectInspector cachedObjectInspector;
 
@@ -82,20 +72,36 @@
     separator = DefaultSeparator;
   }
 
-  public void initialize(Configuration job, Properties tbl) throws SerDeException {
-    separator = DefaultSeparator;
-    String alt_sep = tbl.getProperty(Constants.SERIALIZATION_FORMAT);
-    if(alt_sep != null && alt_sep.length() > 0) {
+  private String getByteValue(String altValue, String defaultVal) {
+    if (altValue != null && altValue.length() > 0) {
       try {
         byte b [] = new byte[1];
-        b[0] = Byte.valueOf(alt_sep).byteValue();
-        separator = new String(b);
+        b[0] = Byte.valueOf(altValue).byteValue();
+        return new String(b);
       } catch(NumberFormatException e) {
-        separator = alt_sep;
+        return altValue;
       }
     }
+    return defaultVal;
+  }
+
+  public void initialize(Configuration job, Properties tbl) throws SerDeException {
+    String alt_sep = tbl.getProperty(Constants.SERIALIZATION_FORMAT);
+    separator = getByteValue(alt_sep, DefaultSeparator);
+
+    String alt_null = tbl.getProperty(Constants.SERIALIZATION_NULL_FORMAT);
+    nullString = getByteValue(alt_null, defaultNullString);
+
     String columnProperty = tbl.getProperty("columns");
-    if (columnProperty == null || columnProperty.length() == 0) {
+    String serdeName = tbl.getProperty(Constants.SERIALIZATION_LIB);
+    // tables that were serialized with columnsetSerDe doesn't have metadata 
+    // so this hack applies to all such tables 
+    boolean columnsetSerDe = false;
+    if ((serdeName != null) && serdeName.equals("org.apache.hadoop.hive.serde.thrift.columnsetSerDe")) {
+      columnsetSerDe = true;
+    }
+    if (columnProperty == null || columnProperty.length() == 0 
+        || columnsetSerDe) {
       // Hack for tables with no columns
       // Treat it as a table with a single column called "col" 
       cachedObjectInspector = ObjectInspectorFactory.getReflectionObjectInspector(
@@ -104,9 +110,9 @@
       columnNames = Arrays.asList(columnProperty.split(","));
       cachedObjectInspector = MetadataListStructObjectInspector.getInstance(columnNames);
     }
-    LOG.info(getClass().getName() + ": initialized with columnNames: " + columnNames );
+    LOG.debug(getClass().getName() + ": initialized with columnNames: " + columnNames + " and separator code=" + (int)separator.charAt(0) );
   }
-
+  
   public static Object deserialize(ColumnSet c, String row, String sep, String nullString) throws Exception {
     if (c.col == null) {
       c.col = new ArrayList<String>();

Modified: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java?rev=706704&r1=706703&r2=706704&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java (original)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java Tue Oct 21 11:11:05 2008
@@ -170,7 +170,7 @@
     switch(oi.getCategory()) {
       case PRIMITIVE: {
         if (o == null) {
-          sb.append("\\N");
+          sb.append("null");
         } else if (o instanceof String) {
           sb.append(QUOTE);
           sb.append(escapeString((String)o));
@@ -184,54 +184,66 @@
         break;
       }
       case LIST: {
-        sb.append(LBRACKET);
         ListObjectInspector loi = (ListObjectInspector)oi;
         ObjectInspector listElementObjectInspector = loi.getListElementObjectInspector();
         List<?> olist = loi.getList(o);
-        for (int i=0; i<olist.size(); i++) {
-          if (i>0) sb.append(COMMA);
-          buildJSONString(sb, olist.get(i), listElementObjectInspector);
+        if (olist == null) {
+          sb.append("null");
+        } else {
+          sb.append(LBRACKET);
+          for (int i=0; i<olist.size(); i++) {
+            if (i>0) sb.append(COMMA);
+            buildJSONString(sb, olist.get(i), listElementObjectInspector);
+          }
+          sb.append(RBRACKET);
         }
-        sb.append(RBRACKET);
         break;
       }
       case MAP: {
-        sb.append(LBRACE);
         MapObjectInspector moi = (MapObjectInspector)oi;
         ObjectInspector mapKeyObjectInspector = moi.getMapKeyObjectInspector();
         ObjectInspector mapValueObjectInspector = moi.getMapValueObjectInspector();
         Map<?,?> omap = moi.getMap(o);
-        boolean first = true;
-        for(Object entry : omap.entrySet()) {
-          if (first) {
-            first = false;
-          } else {
-            sb.append(COMMA);
+        if (omap == null) {
+          sb.append("null");
+        } else {
+          sb.append(LBRACE);
+          boolean first = true;
+          for(Object entry : omap.entrySet()) {
+            if (first) {
+              first = false;
+            } else {
+              sb.append(COMMA);
+            }
+            Map.Entry<?,?> e = (Map.Entry<?,?>)entry;
+            buildJSONString(sb, e.getKey(), mapKeyObjectInspector);
+            sb.append(COLON);
+            buildJSONString(sb, e.getValue(), mapValueObjectInspector);
           }
-          Map.Entry<?,?> e = (Map.Entry<?,?>)entry;
-          buildJSONString(sb, e.getKey(), mapKeyObjectInspector);
-          sb.append(COLON);
-          buildJSONString(sb, e.getValue(), mapValueObjectInspector);
+          sb.append(RBRACE);
         }
-        sb.append(RBRACE);
         break;
       }
       case STRUCT: {
-        sb.append(LBRACE);
         StructObjectInspector soi = (StructObjectInspector)oi;
         List<? extends StructField> structFields = soi.getAllStructFieldRefs();
-        for(int i=0; i<structFields.size(); i++) {
-          if (i>0) {
-            sb.append(COMMA);
+        if (structFields == null) {
+          sb.append("null");
+        } else {
+          sb.append(LBRACE);
+          for(int i=0; i<structFields.size(); i++) {
+            if (i>0) {
+              sb.append(COMMA);
+            }
+            sb.append(QUOTE);
+            sb.append(structFields.get(i).getFieldName());
+            sb.append(QUOTE);
+            sb.append(COLON);
+            buildJSONString(sb, soi.getStructFieldData(o, structFields.get(i)), 
+                structFields.get(i).getFieldObjectInspector());          
           }
-          sb.append(QUOTE);
-          sb.append(structFields.get(i).getFieldName());
-          sb.append(QUOTE);
-          sb.append(COLON);
-          buildJSONString(sb, soi.getStructFieldData(o, structFields.get(i)), 
-              structFields.get(i).getFieldObjectInspector());          
+          sb.append(RBRACE);
         }
-        sb.append(RBRACE);
         break;
       }
       default:

Modified: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/Serializer.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/Serializer.java?rev=706704&r1=706703&r2=706704&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/Serializer.java (original)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/Serializer.java Tue Oct 21 11:11:05 2008
@@ -54,5 +54,4 @@
    */
   public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException;
 
-  public String getShortName();
 }

Modified: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/ThriftByteStreamTypedSerDe.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/ThriftByteStreamTypedSerDe.java?rev=706704&r1=706703&r2=706704&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/ThriftByteStreamTypedSerDe.java (original)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/ThriftByteStreamTypedSerDe.java Tue Oct 21 11:11:05 2008
@@ -22,9 +22,7 @@
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Writable;
 
 import com.facebook.thrift.TBase;
@@ -48,15 +46,6 @@
     throw new SerDeException("ThriftByteStreamTypedSerDe is still semi-abstract");
   }
 
-  public static String shortName() {
-    return "thriftbytestream";
-  }
-
-  public  String getShortName() {
-    return shortName();
-  }
-
-
   public ThriftByteStreamTypedSerDe(Type objectType, TProtocolFactory inFactory,
                                     TProtocolFactory outFactory) throws SerDeException {
     super(objectType);

Modified: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/ThriftDeserializer.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/ThriftDeserializer.java?rev=706704&r1=706703&r2=706704&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/ThriftDeserializer.java (original)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/ThriftDeserializer.java Tue Oct 21 11:11:05 2008
@@ -28,26 +28,6 @@
 
 public class ThriftDeserializer implements Deserializer {
 
-  public static String shortName() {
-    return "thrift";
-  }
-
-  public  String getShortName() {
-    return shortName();
-  }
-
-  static {
-    StackTraceElement[] sTrace = new Exception().getStackTrace();
-    String className = sTrace[0].getClassName();
-    try {
-      SerDeUtils.registerSerDe(shortName(), Class.forName(className));
-      // For backward compatibility: this class replaces the following class.
-      SerDeUtils.registerSerDe("org.apache.hadoop.hive.serde.thrift.ThriftSerDe", Class.forName(className));
-    } catch(Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-  
   private ThriftByteStreamTypedSerDe tsd;
 
   public ThriftDeserializer() { }

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDe.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDe.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDe.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDe.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+
+import org.apache.hadoop.hive.serde2.thrift.ConfigurableTProtocol;
+
+import java.util.*;
+import java.io.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.util.StringUtils;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+public class DynamicSerDe implements SerDe, Serializable {
+
+  public static final Log LOG = LogFactory.getLog(DynamicSerDe.class.getName());
+
+  private String type_name;
+  private DynamicSerDeStructBase bt;
+
+  public static final String META_TABLE_NAME = "name";
+
+  transient private thrift_grammar parse_tree;
+  transient protected ByteStream.Input bis_;
+  transient protected ByteStream.Output bos_;
+
+  /**
+   * protocols are protected in case any of their properties need to be queried from another
+   * class in this package. For TCTLSeparatedProtocol for example, may want to query the separators.
+   */
+  transient protected TProtocol oprot_;
+  transient protected TProtocol iprot_;
+
+  TIOStreamTransport tios;
+
+  public void initialize(Configuration job, Properties tbl) throws SerDeException {
+    try {
+
+      String ddl = tbl.getProperty(Constants.SERIALIZATION_DDL);
+      type_name = tbl.getProperty(META_TABLE_NAME);
+      String protoName = tbl.getProperty(Constants.SERIALIZATION_FORMAT);
+
+      if(protoName == null) {
+        protoName = "com.facebook.thrift.protocol.TBinaryProtocol";
+      }
+      TProtocolFactory protFactory = TReflectionUtils.getProtocolFactoryByName(protoName);
+      bos_ = new ByteStream.Output();
+      bis_ = new ByteStream.Input();
+      tios = new TIOStreamTransport(bis_,bos_);
+
+      oprot_ = protFactory.getProtocol(tios);
+      iprot_ = protFactory.getProtocol(tios);
+
+      /**
+       * initialize the protocols
+       */
+
+      if(oprot_ instanceof org.apache.hadoop.hive.serde2.thrift.ConfigurableTProtocol) {
+        ((ConfigurableTProtocol)oprot_).initialize(job, tbl);
+      }
+
+      if(iprot_ instanceof org.apache.hadoop.hive.serde2.thrift.ConfigurableTProtocol) {
+        ((ConfigurableTProtocol)iprot_).initialize(job, tbl);
+      }
+
+      // in theory the include path should come from the configuration
+      List<String> include_path = new ArrayList<String>();
+      include_path.add(".");
+      LOG.debug("ddl=" + ddl);
+      this.parse_tree = new thrift_grammar(new ByteArrayInputStream(ddl.getBytes()), include_path,false);
+      this.parse_tree.Start();
+
+      this.bt = (DynamicSerDeStructBase)this.parse_tree.types.get(type_name);
+
+      if(this.bt == null) {
+        this.bt = (DynamicSerDeStructBase)this.parse_tree.tables.get(type_name);
+      }
+
+      if(this.bt == null) {
+        throw new SerDeException("Could not lookup table type " + type_name + " in this ddl: " + ddl);
+      }
+
+      this.bt.initialize();
+    } catch (Exception e) {
+      System.out.println(StringUtils.stringifyException(e));
+      throw new SerDeException(e);
+    }
+  }
+
+  Object deserializeReuse = null;
+  public Object deserialize(Writable field) throws SerDeException {
+    try {
+      if (field instanceof Text) {
+        Text b = (Text)field;
+        bis_.reset(b.getBytes(), b.getLength());
+      } else {
+        BytesWritable b = (BytesWritable)field;
+        bis_.reset(b.get(), b.getSize());
+      }
+      deserializeReuse = this.bt.deserialize(deserializeReuse, iprot_);
+      return deserializeReuse;
+    } catch(Exception e) {
+      e.printStackTrace();
+      throw new SerDeException(e);
+    }
+  }
+
+  public static ObjectInspector dynamicSerDeStructBaseToObjectInspector(DynamicSerDeTypeBase bt) throws SerDeException {
+    if (bt.isList()) {
+      return ObjectInspectorFactory.getStandardListObjectInspector(
+          dynamicSerDeStructBaseToObjectInspector(((DynamicSerDeTypeList)bt).getElementType()));
+    } else if (bt.isMap()) {
+      DynamicSerDeTypeMap btMap = (DynamicSerDeTypeMap)bt; 
+      return ObjectInspectorFactory.getStandardMapObjectInspector(
+          dynamicSerDeStructBaseToObjectInspector(btMap.getKeyType()), 
+          dynamicSerDeStructBaseToObjectInspector(btMap.getValueType()));
+    } else if (bt.isPrimitive()) {
+      return ObjectInspectorFactory.getStandardPrimitiveObjectInspector(bt.getRealType());
+    } else {
+      // Must be a struct
+      DynamicSerDeStructBase btStruct = (DynamicSerDeStructBase)bt;
+      DynamicSerDeFieldList fieldList = btStruct.getFieldList();
+      DynamicSerDeField[] fields = fieldList.getChildren();
+      ArrayList<String> fieldNames = new ArrayList<String>(fields.length); 
+      ArrayList<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(fields.length); 
+      for(int i=0; i<fields.length; i++) {
+        fieldNames.add(fields[i].name);
+        fieldObjectInspectors.add(
+            dynamicSerDeStructBaseToObjectInspector(fields[i].getFieldType().getMyType()));
+      }
+      return ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, fieldObjectInspectors);
+    }
+  }
+
+  @Override
+  public ObjectInspector getObjectInspector() throws SerDeException {
+    return dynamicSerDeStructBaseToObjectInspector(this.bt);
+  }
+
+  @Override
+  public Class<? extends Writable> getSerializedClass() {
+    return BytesWritable.class;
+  }
+
+  BytesWritable ret = new BytesWritable();
+  @Override
+  public Writable serialize(Object obj, ObjectInspector objInspector)
+  throws SerDeException {
+    try {
+      bos_.reset();
+      this.bt.serialize(obj, objInspector, oprot_);
+      oprot_.getTransport().flush();
+    } catch(Exception e) {
+      e.printStackTrace();
+      throw new SerDeException(e);
+    }
+    ret.set(bos_.getData(),0,bos_.getCount());
+    return ret;
+  }
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeField.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeField.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeField.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeField.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import org.apache.hadoop.hive.serde2.*;
+import com.facebook.thrift.TException;
+import com.facebook.thrift.protocol.TProtocol;
+
+public class DynamicSerDeField extends DynamicSerDeSimpleNode {
+
+
+  // production is:
+  // [this.fieldid :] Requiredness() FieldType() this.name FieldValue() [CommaOrSemicolon()]
+
+  private final int FD_REQUIREDNESS = 0;
+  private final int FD_FIELD_TYPE = 1;
+  private final int FD_FIELD_VALUE =2;
+
+  public DynamicSerDeFieldType getFieldType() {
+    return (DynamicSerDeFieldType)this.jjtGetChild(FD_FIELD_TYPE);
+  }
+
+  public DynamicSerDeField(int i) {
+    super(i);
+  }
+  public DynamicSerDeField(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+}
+

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFieldList.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFieldList.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFieldList.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFieldList.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,214 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.TApplicationException;
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.server.*;
+import com.facebook.thrift.transport.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+
+import java.lang.reflect.*;
+import com.facebook.thrift.protocol.TType.*;
+
+public class DynamicSerDeFieldList extends DynamicSerDeSimpleNode implements Serializable {
+
+  //  private void writeObject(ObjectOutputStream out) throws IOException {
+  //    out.writeObject(types_by_column_name);
+  //    out.writeObject(ordered_types);
+  //  }
+
+  // production: Field()*
+
+  // mapping of the fieldid to the field
+  private Map<Integer,DynamicSerDeTypeBase> types_by_id = null;
+  private Map<String,DynamicSerDeTypeBase> types_by_column_name = null;
+  private DynamicSerDeTypeBase ordered_types[] = null;
+
+  private Map<String,Integer> ordered_column_id_by_name = null;
+
+  public DynamicSerDeFieldList(int i) {
+    super(i);
+  }
+
+  public DynamicSerDeFieldList(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  private DynamicSerDeField getField(int i) {
+    return (DynamicSerDeField)this.jjtGetChild(i);
+  }
+
+  final public DynamicSerDeField [] getChildren() {
+    int size = this.jjtGetNumChildren();
+    DynamicSerDeField result [] = new DynamicSerDeField[size];
+    for(int i = 0; i < size; i++) {
+      result[i] = (DynamicSerDeField)this.jjtGetChild(i);
+    }
+    return result;
+  }
+
+  private int getNumFields() {
+    return this.jjtGetNumChildren();
+  }
+
+  public void initialize() {
+    if(types_by_id == null) {
+      // multiple means of lookup
+      types_by_id = new HashMap<Integer, DynamicSerDeTypeBase> ();
+      types_by_column_name = new HashMap<String, DynamicSerDeTypeBase> ();
+      ordered_types = new DynamicSerDeTypeBase[this.jjtGetNumChildren()];
+      ordered_column_id_by_name = new HashMap<String, Integer>();
+
+      // put them in and also roll them up while we're at it
+      // a Field contains a FieldType which in turn contains a type
+      for(int i = 0 ; i < this.jjtGetNumChildren(); i++) {
+        DynamicSerDeField mt = this.getField(i);
+        DynamicSerDeTypeBase type = mt.getFieldType().getMyType();
+        type.fieldid = mt.fieldid;
+        type.name = mt.name;
+
+        types_by_id.put(Integer.valueOf(mt.fieldid) , type);
+        types_by_column_name.put(mt.name, type);
+        ordered_types[i] = type;
+        ordered_column_id_by_name.put(mt.name, i);
+      }
+    }
+  }
+
+  private DynamicSerDeTypeBase getFieldByFieldId(int i) {
+    return types_by_id.get(i);
+  }
+
+  protected DynamicSerDeTypeBase getFieldByName(String fieldname) {
+    return types_by_column_name.get(fieldname);
+  }
+
+
+  public Object deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    ArrayList<Object> struct = null;
+
+    if (reuse == null) {
+      struct = new ArrayList<Object>(this.getNumFields());
+      for(int i=0; i<this.getNumFields(); i++) {
+        struct.add(null);
+      }
+    } else {
+      struct = (ArrayList<Object>) reuse;
+      assert(struct.size() == this.getNumFields());
+    }
+
+    // Read the fields.
+    for(int i = 0; i < this.getNumFields(); i++) {
+      DynamicSerDeTypeBase mt = null;
+      TField field = null;
+
+      if (thrift_mode) {
+        field = iprot.readFieldBegin();
+
+        if(field.type >= 0) {
+          if(field.type == TType.STOP) {
+            break;
+          }
+          mt = this.getFieldByFieldId(field.id);
+          if(mt == null) {
+            System.err.println("ERROR for fieldid: " + field.id + " system has no knowledge of this field which is of type : " + field.type);
+            TProtocolUtil.skip(iprot,field.type);
+            continue;
+          }
+        }
+      }
+
+      // field.type < 0 means that this is a faked Thrift field, e.g., TControlSeparatedProtocol, which does not
+      // serialize the field id in the stream.  As a result, the only way to get the field id is to fall back to 
+      // the position "i".
+      // The intention of this hack (field.type < 0) is to make TControlSeparatedProtocol a real Thrift prototype,
+      // but there are a lot additional work to do to fulfill that, and that protocol inherently does not support
+      // versioning (adding/deleting fields).
+      int orderedId = -1;
+      if (!thrift_mode || field.type < 0) {
+        mt = this.ordered_types[i];
+        // We don't need to lookup order_column_id_by_name because we know it must be "i".
+        orderedId = i;
+      } else {
+        // Set the correct position
+        orderedId = ordered_column_id_by_name.get(mt.name); 
+      }
+      struct.set(orderedId, mt.deserialize(struct.get(orderedId), iprot));
+
+      if(thrift_mode) {
+        iprot.readFieldEnd();
+      }
+    }
+    return struct;
+  }
+
+
+  TField field = new TField();
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+
+    // Assuming the ObjectInspector represents exactly the same type as this struct.
+    // This assumption should be checked during query compile time.
+    assert(oi instanceof StructObjectInspector);
+    StructObjectInspector soi = (StructObjectInspector) oi;
+
+    // For every field
+    List<? extends StructField> fields = soi.getAllStructFieldRefs();
+    if (fields.size() != ordered_types.length) {
+      throw new SerDeException("Trying to serialize " + fields.size() 
+          + " fields into a struct with " + ordered_types.length);
+    }
+    for (int i=0; i<fields.size(); i++) {
+      Object f = soi.getStructFieldData(o, fields.get(i));
+      DynamicSerDeTypeBase mt = ordered_types[i];
+      
+      if(thrift_mode) {
+        field.name = mt.name;
+        field.type = mt.getType();
+        field.id = (short)mt.fieldid;
+        oprot.writeFieldBegin(field);
+      }
+
+      mt.serialize(f, fields.get(i).getFieldObjectInspector(), oprot);
+
+      if(thrift_mode) {
+        oprot.writeFieldEnd();
+      }
+    }
+    if(thrift_mode) {
+      oprot.writeFieldStop();
+    }
+  }
+
+  public String toString() {
+    StringBuffer result = new StringBuffer();
+    String prefix = "";
+    for(DynamicSerDeField t: this.getChildren()) {
+      result.append(prefix + t.fieldid + ":" + t.getFieldType().getMyType().toString() + " " + t.name);
+      prefix = ",";
+    }
+    return result.toString();
+  }
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFieldType.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFieldType.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFieldType.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFieldType.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+// basically just a container for the real type so more like a proxy
+public class DynamicSerDeFieldType extends DynamicSerDeSimpleNode {
+
+  // production: this.name | BaseType() | MapType() | SetType() | ListType()
+
+  private final int FD_FIELD_TYPE = 0;
+  public DynamicSerDeFieldType(int i) {
+    super(i);
+  }
+  public DynamicSerDeFieldType(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  protected DynamicSerDeTypeBase getMyType() {
+    // bugbug, need to deal with a named type here - i.e., look it up and proxy to it
+    // should raise an exception if this is a typedef since won't be any children
+    // and thus we can quickly find this comment and limitation.
+    return (DynamicSerDeTypeBase)this.jjtGetChild(FD_FIELD_TYPE);
+  }
+
+
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFunction.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFunction.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFunction.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeFunction.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.protocol.TMessageType;
+
+public class DynamicSerDeFunction extends DynamicSerDeStructBase {
+
+  // production is: Async() FunctionType() NAME FieldList() Throws() [CommaOrSemicolon]
+
+  private final int FD_ASYNC = 0;
+  private final int FD_FUNCTION_TYPE = 1;
+  private final int FD_FIELD_LIST = 2;
+  private final int FD_THROWS = 3;
+
+  public DynamicSerDeFunction(int i) {
+    super(i);
+  }
+  public DynamicSerDeFunction(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public DynamicSerDeFieldList getFieldList() {
+    return (DynamicSerDeFieldList)this.jjtGetChild(FD_FIELD_LIST);
+  }
+
+  public String toString() {
+    String result = "function " + this.name + " (";
+    result += this.getFieldList().toString();
+    result += ")";
+    return result;
+  }
+
+  public byte getType() {
+    return TMessageType.CALL;
+  }
+
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeSimpleNode.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeSimpleNode.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeSimpleNode.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeSimpleNode.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+public class DynamicSerDeSimpleNode extends SimpleNode {
+  protected static final boolean thrift_mode = true;
+  public DynamicSerDeSimpleNode(int i) {
+    super(i);
+  }
+  public DynamicSerDeSimpleNode(thrift_grammar p, int i) {
+    super(p,i);
+  }
+  protected int fieldid;
+  protected String name;
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeStruct.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeStruct.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeStruct.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeStruct.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeStruct extends DynamicSerDeStructBase {
+
+  // production is: struct this.name { FieldList() }
+
+  final private static int FD_FIELD_LIST = 0;
+
+  public DynamicSerDeStruct(int i) {
+    super(i);
+  }
+  public DynamicSerDeStruct(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public String toString() {
+    String result = "struct " + this.name + "(";
+    result += this.getFieldList().toString();
+    result += ")";
+    return result;
+  }
+
+  public DynamicSerDeFieldList getFieldList() {
+    return (DynamicSerDeFieldList)this.jjtGetChild(FD_FIELD_LIST);
+  }
+
+  public byte getType() {
+    return TType.STRUCT;
+  }
+
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeStructBase.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeStructBase.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeStructBase.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeStructBase.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.protocol.*;
+import java.io.*;
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+abstract public class DynamicSerDeStructBase extends DynamicSerDeTypeBase implements Serializable {
+
+  DynamicSerDeFieldList fieldList;
+
+  public DynamicSerDeStructBase(int i) {
+    super(i);
+  }
+  public DynamicSerDeStructBase(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  abstract public DynamicSerDeFieldList getFieldList();
+
+  public void initialize() {
+    fieldList = getFieldList();
+    fieldList.initialize();
+  }
+
+  public boolean isPrimitive() { return false; }
+
+
+  public Class getRealType() {
+    return List.class;
+  }
+
+  public Object deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    if(thrift_mode) {
+      iprot.readStructBegin();
+    }
+
+    Object o = fieldList.deserialize(reuse, iprot);
+
+    if(thrift_mode) {
+      iprot.readStructEnd();
+    }
+    return o;
+  }
+
+  /**
+   * serialize
+   *
+   * The way to serialize a Thrift "table" which in thrift land is really a function and thus this class's name.
+   *
+   * @param o - this list should be in the order of the function's params for now. If we wanted to remove this requirement,
+   * we'd need to make it a List<Pair<String, Object>> with the String being the field name.
+   *
+   */
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    if(thrift_mode) {
+      oprot.writeStructBegin(new TStruct(this.name));
+    }
+
+    fieldList.serialize(o, oi, oprot);
+
+    if(thrift_mode) {
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+  }
+
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeBase.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeBase.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeBase.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeBase.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import com.facebook.thrift.protocol.TProtocol;
+import com.facebook.thrift.TException;
+import java.io.Serializable;
+
+public abstract class DynamicSerDeTypeBase extends DynamicSerDeSimpleNode implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  public DynamicSerDeTypeBase(int i) {
+    super(i);
+  }
+
+  public DynamicSerDeTypeBase(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public Class getRealType() throws SerDeException {
+    throw new SerDeException("Not implemented in base");
+  }
+
+  public Object get(Object obj) {
+    throw new RuntimeException("Not implemented in base");
+  }
+
+  public abstract Object deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException;
+
+  public abstract void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException;
+
+  public String toString() {
+    return "BAD";
+  }
+
+  public byte getType() {
+    return -1;
+  }
+
+  public boolean isPrimitive() { return true; }
+  public boolean isList() { return false; }
+  public boolean isMap() { return false; }
+
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeBool.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeBool.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeBool.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeBool.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.protocol.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeTypeBool extends DynamicSerDeTypeBase {
+
+  // production is: bool
+
+  public DynamicSerDeTypeBool(int i) {
+    super(i);
+  }
+
+  public DynamicSerDeTypeBool(thrift_grammar p, int i) {
+    super(p, i);
+  }
+
+  public String toString() {
+    return "bool";
+  }
+
+  @Override
+  public Object deserialize(Object reuse, TProtocol iprot)
+      throws SerDeException, TException, IllegalAccessException {
+    return Boolean.valueOf(iprot.readBool());
+  }
+
+  @Override
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot)
+      throws TException, SerDeException, NoSuchFieldException,
+      IllegalAccessException {
+    assert (oi.getCategory() == ObjectInspector.Category.PRIMITIVE);
+    assert (((PrimitiveObjectInspector) oi).getPrimitiveClass()
+        .equals(Boolean.class));
+    oprot.writeBool((Boolean) o);
+  }
+
+  public byte getType() {
+    return TType.BOOL;
+  }
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeByte.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeByte.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeByte.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeByte.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.protocol.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeTypeByte extends DynamicSerDeTypeBase {
+
+  // production is: byte
+
+
+  public DynamicSerDeTypeByte(int i) {
+    super(i);
+  }
+  public DynamicSerDeTypeByte(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public String toString() { return "byte"; }
+
+  public Byte deserialize(TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    return Byte.valueOf(iprot.readByte());
+  }
+  public Object deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    return Byte.valueOf(iprot.readByte());
+  }
+
+  public void serialize(Object s, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    // bugbug need to use object of byte type!!!
+    oprot.writeByte((Byte)s);
+  }
+
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    assert(oi.getCategory() == ObjectInspector.Category.PRIMITIVE);
+    assert(((PrimitiveObjectInspector)oi).getPrimitiveClass().equals(Byte.class));
+    oprot.writeByte((Byte)o);
+  }
+
+  public byte getType() {
+    return TType.BYTE;
+  }
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeDouble.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeDouble.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeDouble.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeDouble.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.TApplicationException;
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.server.*;
+import com.facebook.thrift.transport.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+import java.lang.reflect.*;
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeTypeDouble extends DynamicSerDeTypeBase {
+
+  // production is: double
+
+  public DynamicSerDeTypeDouble(int i) {
+    super(i);
+  }
+  public DynamicSerDeTypeDouble(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public String toString() { return "double"; }
+
+  public Object deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    return Double.valueOf(iprot.readDouble());
+  }
+
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    assert(oi.getCategory() == ObjectInspector.Category.PRIMITIVE);
+    assert(((PrimitiveObjectInspector)oi).getPrimitiveClass().equals(Double.class));
+    oprot.writeDouble((Double)o);
+  }
+
+  public byte getType() {
+    return TType.DOUBLE;
+  }
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeList.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeList.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeList.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeList.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.protocol.*;
+import java.util.*;
+import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeTypeList extends DynamicSerDeTypeBase {
+
+  public boolean isPrimitive() { return false; }
+  public boolean isList() { return true; }
+
+  // production is: list<FieldType()>
+
+  static final private int FD_TYPE = 0;
+
+  public Class getRealType() {
+    return java.util.ArrayList.class;
+  }
+
+  public DynamicSerDeTypeList(int i) {
+    super(i);
+  }
+  public DynamicSerDeTypeList(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public DynamicSerDeTypeBase getElementType() {
+    return (DynamicSerDeTypeBase)((DynamicSerDeFieldType)this.jjtGetChild(FD_TYPE)).getMyType();
+  }
+
+  public String toString() {
+    return Constants.LIST_TYPE_NAME + "<" + this.getElementType().toString()  + ">";
+  }
+
+  @Override
+  public ArrayList<Object> deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    TList thelist = iprot.readListBegin();
+    ArrayList<Object> deserializeReuse;
+    if (reuse != null) {
+      deserializeReuse = (ArrayList<Object>)reuse;
+      // Trim to the size needed
+      while (deserializeReuse.size() > thelist.size) {
+        deserializeReuse.remove(deserializeReuse.size()-1);
+      }
+    } else {
+      deserializeReuse = new ArrayList<Object>();
+    }
+    deserializeReuse.ensureCapacity(thelist.size);
+    for(int i = 0; i < thelist.size; i++) {
+      if (i+1 > deserializeReuse.size()) {
+        deserializeReuse.add(this.getElementType().deserialize(null, iprot));
+      } else {
+        deserializeReuse.set(i, 
+            this.getElementType().deserialize(deserializeReuse.get(i), iprot));
+      }
+    }
+    // in theory, the below call isn't needed in non thrift_mode, but let's not get too crazy
+    iprot.readListEnd();
+    return deserializeReuse;
+  }
+
+  @Override
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    ListObjectInspector loi = (ListObjectInspector)oi;
+    ObjectInspector elementObjectInspector = loi.getListElementObjectInspector();
+    DynamicSerDeTypeBase mt = this.getElementType();
+
+    if (o instanceof List) {
+      List<?> list = (List<?>)o;
+      oprot.writeListBegin(new TList(mt.getType(),list.size()));
+      for (Object element: list) {
+        mt.serialize(element, elementObjectInspector, oprot);
+      }
+    } else {
+      Object[] list = (Object[])o;
+      oprot.writeListBegin(new TList(mt.getType(),list.length));
+      for (Object element: list) {
+        mt.serialize(element, elementObjectInspector, oprot);
+      }
+    }
+    // in theory, the below call isn't needed in non thrift_mode, but let's not get too crazy
+    oprot.writeListEnd();
+  }
+
+  public byte getType() {
+    return TType.LIST;
+  }
+
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeMap.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeMap.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeMap.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeMap.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.TApplicationException;
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.server.*;
+import com.facebook.thrift.transport.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import java.lang.reflect.*;
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeTypeMap extends DynamicSerDeTypeBase {
+
+  public boolean isPrimitive() { return false; }
+  public boolean isMap() { return true;}
+
+  // production is: Map<FieldType(),FieldType()>
+
+  private final byte FD_KEYTYPE = 0;
+  private final byte FD_VALUETYPE = 1;
+
+  // returns Map<?,?>
+  public Class getRealType() {
+    try {
+      Class c = this.getKeyType().getRealType();
+      Class c2 = this.getValueType().getRealType();
+      Object o = c.newInstance();
+      Object o2 = c2.newInstance();
+      Map<?,?> l = Collections.singletonMap(o,o2);
+      return l.getClass();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw new RuntimeException(e);
+    }
+  }
+
+  public DynamicSerDeTypeMap(int i) {
+    super(i);
+  }
+
+  public DynamicSerDeTypeMap(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public DynamicSerDeTypeBase getKeyType() {
+    return (DynamicSerDeTypeBase)((DynamicSerDeFieldType)this.jjtGetChild(FD_KEYTYPE)).getMyType();
+  }
+
+  public DynamicSerDeTypeBase getValueType() {
+    return (DynamicSerDeTypeBase)((DynamicSerDeFieldType)this.jjtGetChild(FD_VALUETYPE)).getMyType();
+  }
+
+  public String toString() {
+    return "map<" + this.getKeyType().toString() + "," + this.getValueType().toString() + ">";
+  }
+
+  public Map<Object,Object> deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    HashMap<Object, Object> deserializeReuse;
+    if (reuse != null) {
+      deserializeReuse = (HashMap<Object, Object>)reuse;
+      deserializeReuse.clear();
+    } else {
+      deserializeReuse = new HashMap<Object, Object>();
+    }
+    TMap themap = iprot.readMapBegin();
+    for(int i = 0; i < themap.size; i++) {
+      Object key = this.getKeyType().deserialize(null, iprot);
+      Object value = this.getValueType().deserialize(null, iprot);
+      deserializeReuse.put(key,value);
+    }
+
+    // in theory, the below call isn't needed in non thrift_mode, but let's not get too crazy
+    iprot.readMapEnd();
+    return deserializeReuse;
+  }
+
+  @Override
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot)
+  throws TException, SerDeException, NoSuchFieldException,
+  IllegalAccessException {
+    DynamicSerDeTypeBase keyType = this.getKeyType();
+    DynamicSerDeTypeBase valueType = this.getValueType();
+
+    assert(oi.getCategory() == ObjectInspector.Category.MAP);
+    MapObjectInspector moi = (MapObjectInspector)oi;
+    ObjectInspector koi = moi.getMapKeyObjectInspector();
+    ObjectInspector voi = moi.getMapValueObjectInspector();
+
+    Map<?,?> map = moi.getMap(o);
+    oprot.writeMapBegin(new TMap(keyType.getType(),valueType.getType(),map.size()));
+    for(Iterator i = map.entrySet().iterator(); i.hasNext(); ) {
+      Map.Entry it = (Map.Entry)i.next();
+      Object key = it.getKey();
+      Object value = it.getValue();
+      keyType.serialize(key, koi, oprot);
+      valueType.serialize(value, voi, oprot);
+    }
+    // in theory, the below call isn't needed in non thrift_mode, but let's not get too crazy
+    oprot.writeMapEnd();      
+  }
+
+  public byte getType() {
+    return TType.MAP;
+  }
+};
+

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeSet.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeSet.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeSet.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeSet.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.TApplicationException;
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.server.*;
+import com.facebook.thrift.transport.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import java.lang.reflect.*;
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeTypeSet extends DynamicSerDeTypeBase {
+
+  // production is: set<FieldType()>
+
+  static final private int FD_TYPE = 0;
+
+  public DynamicSerDeTypeSet(int i) {
+    super(i);
+  }
+  public DynamicSerDeTypeSet(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  // returns Set<?>
+  public Class getRealType() {
+    try {
+      Class c = this.getElementType().getRealType();
+      Object o = c.newInstance();
+      Set<?> l = Collections.singleton(o);
+      return l.getClass();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw new RuntimeException(e);
+    }
+  }
+
+  public DynamicSerDeTypeBase getElementType() {
+    return (DynamicSerDeTypeBase)((DynamicSerDeFieldType)this.jjtGetChild(FD_TYPE)).getMyType();
+  }
+
+  public String toString() {
+    return "set<" + this.getElementType().toString()  + ">";
+  }
+
+  public byte getType() {
+    return TType.SET;
+  }
+
+  /** NOTE: Set is not supported by Hive yet.
+   */
+  @Override
+  public Object deserialize(Object reuse, TProtocol iprot)
+  throws SerDeException, TException, IllegalAccessException {
+    TSet theset = iprot.readSetBegin();
+    Set<Object> result;
+    if (reuse != null) {
+      result = (Set<Object>)reuse;
+      result.clear();
+    } else {
+      result = new HashSet<Object>();
+    }
+    for(int i = 0; i < theset.size; i++) {
+      Object elem = this.getElementType().deserialize(null, iprot);
+      result.add(elem);
+    }
+    // in theory, the below call isn't needed in non thrift_mode, but let's not get too crazy
+    iprot.readSetEnd();
+    return result;
+  }
+
+  /** NOTE: Set is not supported by Hive yet.
+   *  The code uses ListObjectInspector right now. We need to change it to 
+   *  SetObjectInspector when that is done.
+   */
+  @Override
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot)
+  throws TException, SerDeException, NoSuchFieldException,
+  IllegalAccessException {
+
+    ListObjectInspector loi = (ListObjectInspector)oi;
+
+    Set<Object> set = (Set<Object>)o;
+    DynamicSerDeTypeBase mt = this.getElementType();
+    oprot.writeSetBegin(new TSet(mt.getType(),set.size()));
+    for(Object element: set) {
+      mt.serialize(element, loi.getListElementObjectInspector(), oprot);
+    }
+    // in theory, the below call isn't needed in non thrift_mode, but let's not get too crazy
+    oprot.writeSetEnd();
+  }
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeString.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeString.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeString.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypeString.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.TApplicationException;
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.server.*;
+import com.facebook.thrift.transport.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+import java.lang.reflect.*;
+import com.facebook.thrift.protocol.TType.*;
+
+public class DynamicSerDeTypeString extends DynamicSerDeTypeBase {
+
+  // production is: string
+
+  public DynamicSerDeTypeString(int i) {
+    super(i);
+  }
+  public DynamicSerDeTypeString(thrift_grammar p, int i) {
+    super(p,i);
+  }
+  public Class getRealType() { return java.lang.String.class; }
+
+  public String toString() { return "string"; }
+
+  public String deserialize(TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    return String.valueOf(iprot.readString());
+  }
+
+  @Override
+  public Object deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    return String.valueOf(iprot.readString());
+  }
+
+  public void serialize(Object s, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    oprot.writeString((String)s);
+  }
+  @Override
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    assert(oi.getCategory() == ObjectInspector.Category.PRIMITIVE);
+    assert(((PrimitiveObjectInspector)oi).getPrimitiveClass().equals(String.class));
+    oprot.writeString((String)o);
+  }
+
+  public byte getType() {
+    return TType.STRING;
+  }
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypedef.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypedef.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypedef.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypedef.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.TApplicationException;
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.server.*;
+import com.facebook.thrift.transport.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import java.lang.reflect.*;
+import com.facebook.thrift.protocol.TType.*;
+
+public class DynamicSerDeTypedef extends DynamicSerDeTypeBase {
+
+  // production is: typedef DefinitionType() this.name
+
+  private final static int FD_DEFINITION_TYPE = 0;
+
+  public DynamicSerDeTypedef(int i) {
+    super(i);
+  }
+  public DynamicSerDeTypedef(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  private DynamicSerDeSimpleNode getDefinitionType() {
+    return (DynamicSerDeSimpleNode)this.jjtGetChild(FD_DEFINITION_TYPE);
+  }
+
+
+  public DynamicSerDeTypeBase getMyType() {
+    DynamicSerDeSimpleNode child = this.getDefinitionType();
+    DynamicSerDeTypeBase ret = (DynamicSerDeTypeBase)child.jjtGetChild(0);
+    return ret;
+  }
+
+  public String toString() {
+    String result = "typedef " + this.name + "(";
+    result += this.getDefinitionType().toString();
+    result += ")";
+    return result;
+  }
+
+  public byte getType() {
+    throw new RuntimeException("not implemented");
+  }
+
+  @Override
+  public Object deserialize(Object reuse, TProtocol iprot)
+  throws SerDeException, TException, IllegalAccessException {
+    throw new RuntimeException("not implemented");
+  }
+  @Override
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot)
+  throws TException, SerDeException, NoSuchFieldException,
+  IllegalAccessException {
+    throw new RuntimeException("not implemented");
+  }
+
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei16.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei16.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei16.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei16.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.TApplicationException;
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.server.*;
+import com.facebook.thrift.transport.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+import java.lang.reflect.*;
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeTypei16 extends DynamicSerDeTypeBase {
+
+  public Class getRealType() { return Integer.valueOf(2).getClass(); }
+
+  // production is: i16
+
+  public DynamicSerDeTypei16(int i) {
+    super(i);
+  }
+  public DynamicSerDeTypei16(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public String toString() { return "i16"; }
+
+  @Override
+  public Object deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    return Integer.valueOf(iprot.readI16());
+  }
+
+  @Override
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    assert(oi.getCategory() == ObjectInspector.Category.PRIMITIVE);
+    assert(((PrimitiveObjectInspector)oi).getPrimitiveClass().equals(Short.class));
+    oprot.writeI16((Short)o);
+  }
+
+  public byte getType() {
+    return TType.I16;
+  }
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei32.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei32.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei32.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei32.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.TApplicationException;
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.server.*;
+import com.facebook.thrift.transport.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+import java.lang.reflect.*;
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeTypei32 extends DynamicSerDeTypeBase {
+
+  // production is: i32
+
+  public DynamicSerDeTypei32(int i) {
+    super(i);
+  }
+  public DynamicSerDeTypei32(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public String toString() { return "i32"; }
+
+  @Override
+  public Object deserialize(Object reuse, TProtocol iprot) throws SerDeException, TException, IllegalAccessException {
+    return Integer.valueOf(iprot.readI32());
+  }
+
+  @Override
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    assert(oi.getCategory() == ObjectInspector.Category.PRIMITIVE);
+    assert(((PrimitiveObjectInspector)oi).getPrimitiveClass().equals(Integer.class));
+    oprot.writeI32((Integer)o);
+  }
+
+  public Class getRealType() { return java.lang.Integer.class; }
+  public Integer getRealTypeInstance() { return Integer.valueOf(0); }
+
+
+  public byte getType() {
+    return TType.I32;
+  }
+}

Added: hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei64.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei64.java?rev=706704&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei64.java (added)
+++ hadoop/core/trunk/src/contrib/hive/serde/src/java/org/apache/hadoop/hive/serde2/dynamic_type/DynamicSerDeTypei64.java Tue Oct 21 11:11:05 2008
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.dynamic_type;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.TApplicationException;
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.server.*;
+import com.facebook.thrift.transport.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.hive.serde2.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+import java.lang.reflect.*;
+import com.facebook.thrift.protocol.TType;
+
+public class DynamicSerDeTypei64 extends DynamicSerDeTypeBase {
+
+  public Class getRealType() { return Long.valueOf(0).getClass(); }
+
+  // production is: i64
+
+  public DynamicSerDeTypei64(int i) {
+    super(i);
+  }
+  public DynamicSerDeTypei64(thrift_grammar p, int i) {
+    super(p,i);
+  }
+
+  public String toString() { return "i64"; }
+
+  @Override
+  public Object deserialize(Object reuse, TProtocol iprot)  throws SerDeException, TException, IllegalAccessException {
+    return Long.valueOf(iprot.readI64());
+  }
+
+  @Override
+  public void serialize(Object o, ObjectInspector oi, TProtocol oprot) throws TException, SerDeException, NoSuchFieldException,IllegalAccessException  {
+    assert(oi.getCategory() == ObjectInspector.Category.PRIMITIVE);
+    assert(((PrimitiveObjectInspector)oi).getPrimitiveClass().equals(Long.class));
+    oprot.writeI64((Long)o);
+  }
+
+  public byte getType() {
+    return TType.I64;
+  }
+}