You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2009/04/15 00:54:45 UTC

svn commit: r764994 [11/12] - in /hadoop/hive/trunk: ./ data/conf/ eclipse-templates/ metastore/src/java/org/apache/hadoop/hive/metastore/ ql/ ql/lib/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/io/ ql/src/java/org...

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaStringObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaStringObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaStringObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaStringObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,43 @@
+/**
+ * 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.objectinspector.primitive;
+
+import org.apache.hadoop.io.Text;
+
+
+/**
+ * A JavaStringObjectInspector inspects a Java String Object.
+ */
+public class JavaStringObjectInspector extends AbstractPrimitiveJavaObjectInspector 
+implements StringObjectInspector{
+
+  JavaStringObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.stringTypeEntry);
+  }
+  
+  @Override
+  public Text getPrimitiveWritableObject(Object o) {
+    return o == null ? null : new Text(((String)o));
+  }
+
+  @Override
+  public String getPrimitiveJavaObject(Object o) {
+    return (String)o;
+  }
+
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaVoidObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaVoidObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaVoidObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaVoidObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,38 @@
+/**
+ * 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.objectinspector.primitive;
+
+import org.apache.hadoop.io.NullWritable;
+
+
+/**
+ * A JavaVoidObjectInspector inspects a Java Void Object.
+ */
+public class JavaVoidObjectInspector extends AbstractPrimitiveJavaObjectInspector 
+implements VoidObjectInspector{
+
+  JavaVoidObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.voidTypeEntry);
+  }
+  
+  @Override
+  public Object getPrimitiveWritableObject(Object o) {
+    return NullWritable.get();
+  }
+
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/LongObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/LongObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/LongObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/LongObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,32 @@
+/**
+ * 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.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+
+/**
+ * A LongObjectInspector inspects an Object representing a Long.
+ */
+public interface LongObjectInspector extends PrimitiveObjectInspector {
+  
+  /**
+   * Get the long data.
+   */
+  long get(Object o);
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,142 @@
+/**
+ * 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.objectinspector.primitive;
+
+import java.util.HashMap;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableBooleanObjectInspector;
+
+
+/**
+ * PrimitiveObjectInspectorFactory is the primary way to create new PrimitiveObjectInspector 
+ * instances.
+ * 
+ * The reason of having caches here is that ObjectInspector is because ObjectInspectors do
+ * not have an internal state - so ObjectInspectors with the same construction parameters should
+ * result in exactly the same ObjectInspector.
+ */
+public class PrimitiveObjectInspectorFactory {
+
+  public final static JavaBooleanObjectInspector javaBooleanObjectInspector= new JavaBooleanObjectInspector();
+  public final static JavaByteObjectInspector javaByteObjectInspector= new JavaByteObjectInspector();
+  public final static JavaShortObjectInspector javaShortObjectInspector= new JavaShortObjectInspector();
+  public final static JavaIntObjectInspector javaIntObjectInspector= new JavaIntObjectInspector();
+  public final static JavaLongObjectInspector javaLongObjectInspector= new JavaLongObjectInspector();
+  public final static JavaFloatObjectInspector javaFloatObjectInspector= new JavaFloatObjectInspector();
+  public final static JavaDoubleObjectInspector javaDoubleObjectInspector= new JavaDoubleObjectInspector();
+  public final static JavaStringObjectInspector javaStringObjectInspector= new JavaStringObjectInspector();
+  public final static JavaVoidObjectInspector javaVoidObjectInspector= new JavaVoidObjectInspector();
+
+  public final static WritableBooleanObjectInspector writableBooleanObjectInspector= new WritableBooleanObjectInspector();
+  public final static WritableByteObjectInspector writableByteObjectInspector= new WritableByteObjectInspector();
+  public final static WritableShortObjectInspector writableShortObjectInspector= new WritableShortObjectInspector();
+  public final static WritableIntObjectInspector writableIntObjectInspector= new WritableIntObjectInspector();
+  public final static WritableLongObjectInspector writableLongObjectInspector= new WritableLongObjectInspector();
+  public final static WritableFloatObjectInspector writableFloatObjectInspector= new WritableFloatObjectInspector();
+  public final static WritableDoubleObjectInspector writableDoubleObjectInspector= new WritableDoubleObjectInspector();
+  public final static WritableStringObjectInspector writableStringObjectInspector= new WritableStringObjectInspector();
+  public final static WritableVoidObjectInspector writableVoidObjectInspector= new WritableVoidObjectInspector();
+  
+  
+  private static HashMap<PrimitiveCategory, AbstractPrimitiveWritableObjectInspector> 
+      cachedPrimitiveWritableInspectorCache = 
+        new HashMap<PrimitiveCategory, AbstractPrimitiveWritableObjectInspector>();
+  static {
+    cachedPrimitiveWritableInspectorCache.put(PrimitiveCategory.BOOLEAN, 
+        writableBooleanObjectInspector);
+    cachedPrimitiveWritableInspectorCache.put(PrimitiveCategory.BYTE, 
+        writableByteObjectInspector);
+    cachedPrimitiveWritableInspectorCache.put(PrimitiveCategory.SHORT, 
+        writableShortObjectInspector);
+    cachedPrimitiveWritableInspectorCache.put(PrimitiveCategory.INT, 
+        writableIntObjectInspector);
+    cachedPrimitiveWritableInspectorCache.put(PrimitiveCategory.LONG, 
+        writableLongObjectInspector);
+    cachedPrimitiveWritableInspectorCache.put(PrimitiveCategory.FLOAT, 
+        writableFloatObjectInspector);
+    cachedPrimitiveWritableInspectorCache.put(PrimitiveCategory.DOUBLE, 
+        writableDoubleObjectInspector);
+    cachedPrimitiveWritableInspectorCache.put(PrimitiveCategory.STRING, 
+        writableStringObjectInspector);
+    cachedPrimitiveWritableInspectorCache.put(PrimitiveCategory.VOID, 
+        writableVoidObjectInspector);
+  }
+  
+
+  private static HashMap<PrimitiveCategory, AbstractPrimitiveJavaObjectInspector> 
+      cachedPrimitiveJavaInspectorCache = 
+        new HashMap<PrimitiveCategory, AbstractPrimitiveJavaObjectInspector>();
+  static {
+    cachedPrimitiveJavaInspectorCache.put(PrimitiveCategory.BOOLEAN, 
+        javaBooleanObjectInspector);
+    cachedPrimitiveJavaInspectorCache.put(PrimitiveCategory.BYTE, 
+        javaByteObjectInspector);
+    cachedPrimitiveJavaInspectorCache.put(PrimitiveCategory.SHORT, 
+        javaShortObjectInspector);
+    cachedPrimitiveJavaInspectorCache.put(PrimitiveCategory.INT, 
+        javaIntObjectInspector);
+    cachedPrimitiveJavaInspectorCache.put(PrimitiveCategory.LONG, 
+        javaLongObjectInspector);
+    cachedPrimitiveJavaInspectorCache.put(PrimitiveCategory.FLOAT, 
+        javaFloatObjectInspector);
+    cachedPrimitiveJavaInspectorCache.put(PrimitiveCategory.DOUBLE, 
+        javaDoubleObjectInspector);
+    cachedPrimitiveJavaInspectorCache.put(PrimitiveCategory.STRING, 
+        javaStringObjectInspector);
+    cachedPrimitiveJavaInspectorCache.put(PrimitiveCategory.VOID, 
+        javaVoidObjectInspector);
+  }
+  
+  
+  /**
+   * Returns the PrimitiveWritableObjectInspector for the PrimitiveCategory.  
+   * @param primitiveCategory
+   */
+  public static AbstractPrimitiveWritableObjectInspector 
+      getPrimitiveWritableObjectInspector(
+          PrimitiveCategory primitiveCategory) {
+    AbstractPrimitiveWritableObjectInspector result = 
+      cachedPrimitiveWritableInspectorCache.get(primitiveCategory);
+    if (result == null) {
+      throw new RuntimeException("Internal error: Cannot find ObjectInspector "
+          + " for " + primitiveCategory);
+    }
+    return result;
+  }
+  
+  /**
+   * Returns the PrimitiveJavaObjectInspector for the PrimitiveCategory.  
+   * @param primitiveCategory
+   */
+  public static AbstractPrimitiveJavaObjectInspector 
+      getPrimitiveJavaObjectInspector(
+          PrimitiveCategory primitiveCategory) {
+    AbstractPrimitiveJavaObjectInspector result = 
+      cachedPrimitiveJavaInspectorCache.get(primitiveCategory);
+    if (result == null) {
+      throw new RuntimeException("Internal error: Cannot find ObjectInspector "
+          + " for " + primitiveCategory);
+    }
+    return result;
+  }
+
+
+  
+  
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,245 @@
+/**
+ * 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.objectinspector.primitive;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+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.io.ByteWritable;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * 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(). 
+ */
+public class PrimitiveObjectInspectorUtils {
+
+  private static Log LOG = LogFactory.getLog(PrimitiveObjectInspectorUtils.class.getName());
+  
+  /**
+   * TypeEntry stores information about a Hive Primitive TypeInfo.
+   */
+  public static class PrimitiveTypeEntry {
+    
+    /**
+     * The category of the PrimitiveType.
+     */
+    public PrimitiveObjectInspector.PrimitiveCategory primitiveCategory;
+    
+    /**
+     * primitiveJavaType refers to java types like int, double, etc. 
+     */
+    public Class<?> primitiveJavaType;
+    /**
+     * primitiveJavaClass refers to java classes like Integer, Double, String etc.
+     */
+    public Class<?> primitiveJavaClass;
+    /**
+     * writableClass refers to hadoop Writable classes like IntWritable, DoubleWritable, Text etc. 
+     */
+    public Class<?> primitiveWritableClass;
+    /**
+     * typeName is the name of the type as in DDL.
+     */
+    public String typeName;
+    
+    PrimitiveTypeEntry(PrimitiveObjectInspector.PrimitiveCategory primitiveCategory, 
+        String typeName, Class<?> primitiveType, Class<?> javaClass, Class<?> hiveClass) {
+      this.primitiveCategory = primitiveCategory;
+      this.primitiveJavaType = primitiveType;
+      this.primitiveJavaClass = javaClass;
+      this.primitiveWritableClass = hiveClass;
+      this.typeName = typeName;
+    }
+  }
+  
+  static final Map<PrimitiveCategory, PrimitiveTypeEntry> primitiveCategoryToTypeEntry = new HashMap<PrimitiveCategory, PrimitiveTypeEntry>();
+  static final Map<Class<?>, PrimitiveTypeEntry> primitiveJavaTypeToTypeEntry = new HashMap<Class<?>, PrimitiveTypeEntry>();
+  static final Map<Class<?>, PrimitiveTypeEntry> primitiveJavaClassToTypeEntry = new HashMap<Class<?>, PrimitiveTypeEntry>();
+  static final Map<Class<?>, PrimitiveTypeEntry> primitiveWritableClassToTypeEntry = new HashMap<Class<?>, PrimitiveTypeEntry>();
+  static final Map<String, PrimitiveTypeEntry> typeNameToTypeEntry = new HashMap<String, PrimitiveTypeEntry>();
+  
+  static void registerType(PrimitiveTypeEntry t) {
+    if (t.primitiveCategory != PrimitiveCategory.UNKNOWN) {
+      primitiveCategoryToTypeEntry.put(t.primitiveCategory, t);
+    }
+    if (t.primitiveJavaType != null) {
+      primitiveJavaTypeToTypeEntry.put(t.primitiveJavaType, t);
+    }
+    if (t.primitiveJavaClass != null) {
+      primitiveJavaClassToTypeEntry.put(t.primitiveJavaClass, t);
+    }
+    if (t.primitiveWritableClass != null) {
+      primitiveWritableClassToTypeEntry.put(t.primitiveWritableClass, t);
+    }
+    if (t.typeName != null) {
+      typeNameToTypeEntry.put(t.typeName, t);
+    }
+  }
+  
+  public static final PrimitiveTypeEntry stringTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.STRING, Constants.STRING_TYPE_NAME, null, String.class, Text.class);
+  public static final PrimitiveTypeEntry booleanTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.BOOLEAN, Constants.BOOLEAN_TYPE_NAME, Boolean.TYPE, Boolean.class, BooleanWritable.class);
+  public static final PrimitiveTypeEntry intTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.INT, Constants.INT_TYPE_NAME, Integer.TYPE, Integer.class, IntWritable.class);
+  public static final PrimitiveTypeEntry longTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.LONG, Constants.BIGINT_TYPE_NAME, Long.TYPE, Long.class, LongWritable.class);
+  public static final PrimitiveTypeEntry floatTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.FLOAT, Constants.FLOAT_TYPE_NAME, Float.TYPE, Float.class, FloatWritable.class);
+  public static final PrimitiveTypeEntry voidTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.VOID, Constants.VOID_TYPE_NAME, Void.TYPE, Void.class, NullWritable.class);
+
+  // No corresponding Writable classes for the following 3 in hadoop 0.17.0
+  public static final PrimitiveTypeEntry doubleTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.DOUBLE, Constants.DOUBLE_TYPE_NAME, Double.TYPE, Double.class, DoubleWritable.class);
+  public static final PrimitiveTypeEntry byteTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.BYTE, Constants.TINYINT_TYPE_NAME, Byte.TYPE, Byte.class, ByteWritable.class);
+  public static final PrimitiveTypeEntry shortTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.SHORT, Constants.SMALLINT_TYPE_NAME, Short.TYPE, Short.class, ShortWritable.class);
+
+  // Following 3 are complex types for special handling
+  public static final PrimitiveTypeEntry unknownTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.UNKNOWN, "unknown", null, Object.class, null);
+  public static final PrimitiveTypeEntry unknownMapTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.UNKNOWN, Constants.MAP_TYPE_NAME, null, Map.class, null);
+  public static final PrimitiveTypeEntry unknownListTypeEntry = new PrimitiveTypeEntry(PrimitiveCategory.UNKNOWN, Constants.LIST_TYPE_NAME, null, List.class, null);
+  
+  static {
+    registerType(stringTypeEntry);
+    registerType(booleanTypeEntry);
+    registerType(intTypeEntry);
+    registerType(longTypeEntry);
+    registerType(floatTypeEntry);
+    registerType(voidTypeEntry);
+    registerType(doubleTypeEntry);
+    registerType(byteTypeEntry);
+    registerType(shortTypeEntry);
+    registerType(unknownTypeEntry);
+    registerType(unknownMapTypeEntry);
+    registerType(unknownListTypeEntry);
+  }
+
+  /**
+   * Return Whether the class is a Java Primitive type or a Java Primitive class. 
+   */
+  public static Class<?> primitiveJavaTypeToClass(Class<?> clazz) {
+    PrimitiveTypeEntry t = primitiveJavaTypeToTypeEntry.get(clazz);
+    return t == null ? clazz : t.primitiveJavaClass;
+  }
+
+  /**
+   * Whether the class is a Java Primitive type or a Java Primitive class. 
+   */
+  public static boolean isPrimitiveJava(Class<?> clazz) {
+    return primitiveJavaTypeToTypeEntry.get(clazz) != null
+           || primitiveJavaClassToTypeEntry.get(clazz) != null;
+  }
+  
+  /**
+   * Whether the class is a Java Primitive type. 
+   */
+  public static boolean isPrimitiveJavaType(Class<?> clazz) {
+    return primitiveJavaTypeToTypeEntry.get(clazz) != null;
+  }
+
+  /**
+   * Whether the class is a Java Primitive class. 
+   */
+  public static boolean isPrimitiveJavaClass(Class<?> clazz) {
+    return primitiveJavaClassToTypeEntry.get(clazz) != null;
+  }
+
+  /**
+   * Whether the class is a Hive Primitive Writable class. 
+   */
+  public static boolean isPrimitiveWritableClass(Class<?> clazz) {
+    return primitiveWritableClassToTypeEntry.get(clazz) != null;
+  }
+  
+  /**
+   * Get the typeName from a Java Primitive Type or Java PrimitiveClass. 
+   */
+  public static String getTypeNameFromPrimitiveJava(Class<?> clazz) {
+    PrimitiveTypeEntry t = primitiveJavaTypeToTypeEntry.get(clazz);
+    if (t == null) {
+      t = primitiveJavaClassToTypeEntry.get(clazz);
+    }
+    return t == null ? null : t.typeName;
+  }
+  
+  /**
+   * Get the typeName from a Primitive Writable Class. 
+   */
+  public static String getTypeNameFromPrimitiveWritable(Class<?> clazz) {
+    PrimitiveTypeEntry t = primitiveWritableClassToTypeEntry.get(clazz);
+    return t == null ? null : t.typeName;
+  }
+
+  /**
+   * Get the typeName from a Java Primitive Type or Java PrimitiveClass. 
+   */
+  public static PrimitiveTypeEntry getTypeEntryFromPrimitiveCategory(PrimitiveCategory category) {
+    return primitiveCategoryToTypeEntry.get(category);
+  }
+  
+  /**
+   * Get the TypeEntry for a Java Primitive Type or Java PrimitiveClass. 
+   */
+  public static PrimitiveTypeEntry getTypeEntryFromPrimitiveJava(Class<?> clazz) {
+    PrimitiveTypeEntry t = primitiveJavaTypeToTypeEntry.get(clazz);
+    if (t == null) {
+      t = primitiveJavaClassToTypeEntry.get(clazz);
+    }
+    return t;
+  }
+  
+  /**
+   * Get the TypeEntry for a Java Primitive Type or Java PrimitiveClass. 
+   */
+  public static PrimitiveTypeEntry getTypeEntryFromPrimitiveJavaType(Class<?> clazz) {
+    return primitiveJavaTypeToTypeEntry.get(clazz);
+  }
+  
+  /**
+   * Get the TypeEntry for a Java Primitive Type or Java PrimitiveClass. 
+   */
+  public static PrimitiveTypeEntry getTypeEntryFromPrimitiveJavaClass(Class<?> clazz) {
+    return primitiveJavaClassToTypeEntry.get(clazz);
+  }
+  
+  /**
+   * Get the TypeEntry for a Primitive Writable Class. 
+   */
+  public static PrimitiveTypeEntry getTypeEntryFromPrimitiveWritableClass(Class<?> clazz) {
+    return primitiveWritableClassToTypeEntry.get(clazz);
+  }
+  
+  /**
+   * Get the TypeEntry for a Primitive Writable Class. 
+   */
+  public static PrimitiveTypeEntry getTypeEntryFromTypeName(String typeName) {
+    return typeNameToTypeEntry.get(typeName);
+  }
+  
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/ShortObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/ShortObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/ShortObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/ShortObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,32 @@
+/**
+ * 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.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+
+/**
+ * A ShortObjectInspector inspects an Object representing a Short.
+ */
+public interface ShortObjectInspector extends PrimitiveObjectInspector {
+  
+  /**
+   * Get the short data.
+   */
+  short get(Object o);
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/StringObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/StringObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/StringObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/StringObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,38 @@
+/**
+ * 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.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.io.Text;
+
+
+/**
+ * A StringObjectInspector inspects an Object representing a String.
+ */
+public interface StringObjectInspector extends PrimitiveObjectInspector {
+  
+  /**
+   * Get the Text representation of the data.
+   */
+  Text getPrimitiveWritableObject(Object o);
+  
+  /**
+   * Get the String representation of the data.
+   */
+  String getPrimitiveJavaObject(Object o);
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/VoidObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/VoidObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/VoidObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/VoidObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,28 @@
+/**
+ * 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.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+
+/**
+ * A VoidObjectInspector can inspect a void object.
+ */
+public interface VoidObjectInspector extends PrimitiveObjectInspector {
+  
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableBooleanObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableBooleanObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableBooleanObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableBooleanObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -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.objectinspector.primitive;
+
+import org.apache.hadoop.io.BooleanWritable;
+
+
+/**
+ * A WritableBooleanObjectInspector inspects a BooleanWritable Object.
+ */
+public class WritableBooleanObjectInspector extends AbstractPrimitiveWritableObjectInspector 
+implements BooleanObjectInspector{
+
+  WritableBooleanObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.booleanTypeEntry);
+  }
+  
+  @Override
+  public boolean get(Object o) {
+    return ((BooleanWritable)o).get();
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return o == null ? null : new BooleanWritable(((BooleanWritable)o).get());
+  }
+
+  @Override
+  public Object getPrimitiveJavaObject(Object o) {
+    return o == null ? null : Boolean.valueOf(((BooleanWritable)o).get());
+  }
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableByteObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableByteObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableByteObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableByteObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -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.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
+
+
+/**
+ * A WritableByteObjectInspector inspects a ByteWritable Object.
+ */
+public class WritableByteObjectInspector extends AbstractPrimitiveWritableObjectInspector 
+implements ByteObjectInspector{
+
+  WritableByteObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.byteTypeEntry);
+  }
+  
+  @Override
+  public byte get(Object o) {
+    return ((ByteWritable)o).get();
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return o == null ? null : new ByteWritable(((ByteWritable)o).get());
+  }
+
+  @Override
+  public Object getPrimitiveJavaObject(Object o) {
+    return o == null ? null : Byte.valueOf(((ByteWritable)o).get());
+  }
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableDoubleObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableDoubleObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableDoubleObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableDoubleObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -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.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+
+
+/**
+ * A WritableDoubleObjectInspector inspects a DoubleWritable Object.
+ */
+public class WritableDoubleObjectInspector extends AbstractPrimitiveWritableObjectInspector 
+implements DoubleObjectInspector{
+
+  WritableDoubleObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.doubleTypeEntry);
+  }
+  
+  @Override
+  public double get(Object o) {
+    return ((DoubleWritable)o).get();
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return o == null ? null : new DoubleWritable(((DoubleWritable)o).get());
+  }
+
+  @Override
+  public Object getPrimitiveJavaObject(Object o) {
+    return o == null ? null : Double.valueOf(((DoubleWritable)o).get());
+  }
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableFloatObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableFloatObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableFloatObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableFloatObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,46 @@
+/**
+ * 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.objectinspector.primitive;
+
+import org.apache.hadoop.io.FloatWritable;
+
+/**
+ * A FloatObjectInspector inspects a FloatWritable Object.
+ */
+public class WritableFloatObjectInspector extends AbstractPrimitiveWritableObjectInspector 
+implements FloatObjectInspector{
+
+  WritableFloatObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.floatTypeEntry);
+  }
+  
+  @Override
+  public float get(Object o) {
+    return ((FloatWritable)o).get();
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return o == null ? null : new FloatWritable(((FloatWritable)o).get());
+  }
+
+  @Override
+  public Object getPrimitiveJavaObject(Object o) {
+    return o == null ? null : Float.valueOf(((FloatWritable)o).get());
+  }
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableIntObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableIntObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableIntObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableIntObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -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.objectinspector.primitive;
+
+import org.apache.hadoop.io.IntWritable;
+
+
+/**
+ * A WritableIntObjectInspector inspects a IntWritable Object.
+ */
+public class WritableIntObjectInspector extends AbstractPrimitiveWritableObjectInspector 
+implements IntObjectInspector{
+
+  WritableIntObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.intTypeEntry);
+  }
+  
+  @Override
+  public int get(Object o) {
+    return ((IntWritable)o).get();
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return o == null ? null : new IntWritable(((IntWritable)o).get());
+  }
+
+  @Override
+  public Object getPrimitiveJavaObject(Object o) {
+    return o == null ? null : Integer.valueOf(((IntWritable)o).get());
+  }
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableLongObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableLongObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableLongObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableLongObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -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.objectinspector.primitive;
+
+import org.apache.hadoop.io.LongWritable;
+
+
+/**
+ * A WritableLongObjectInspector inspects a LongWritable Object.
+ */
+public class WritableLongObjectInspector extends AbstractPrimitiveWritableObjectInspector 
+implements LongObjectInspector{
+
+  WritableLongObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.longTypeEntry);
+  }
+  
+  @Override
+  public long get(Object o) {
+    return ((LongWritable)o).get();
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return o == null ? null : new LongWritable(((LongWritable)o).get());
+  }
+
+  @Override
+  public Object getPrimitiveJavaObject(Object o) {
+    return o == null ? null : Long.valueOf(((LongWritable)o).get());
+  }
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableShortObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableShortObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableShortObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableShortObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -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.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+
+
+/**
+ * A WritableShortObjectInspector inspects a ShortWritable Object.
+ */
+public class WritableShortObjectInspector extends AbstractPrimitiveWritableObjectInspector 
+implements ShortObjectInspector{
+
+  WritableShortObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.shortTypeEntry);
+  }
+  
+  @Override
+  public short get(Object o) {
+    return ((ShortWritable)o).get();
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return o == null ? null : new ShortWritable(((ShortWritable)o).get());
+  }
+
+  @Override
+  public Object getPrimitiveJavaObject(Object o) {
+    return o == null ? null : Short.valueOf(((ShortWritable)o).get());
+  }
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableStringObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableStringObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableStringObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableStringObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.serde2.objectinspector.primitive;
+
+import org.apache.hadoop.io.Text;
+
+
+/**
+ * A WritableStringObjectInspector inspects a Text Object.
+ */
+public class WritableStringObjectInspector extends AbstractPrimitiveWritableObjectInspector 
+implements StringObjectInspector{
+
+  WritableStringObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.stringTypeEntry);
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return new Text((Text)o);
+  }
+
+  @Override
+  public Text getPrimitiveWritableObject(Object o) {
+    return o == null ? null : (Text)o;
+  }
+  
+  @Override
+  public String getPrimitiveJavaObject(Object o) {
+    return o == null ? null : ((Text)o).toString();
+  }
+
+}

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableVoidObjectInspector.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableVoidObjectInspector.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableVoidObjectInspector.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableVoidObjectInspector.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,40 @@
+/**
+ * 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.objectinspector.primitive;
+
+
+/**
+ * A WritableVoidObjectInspector inspects a NullWritable Object.
+ */
+public class WritableVoidObjectInspector extends AbstractPrimitiveWritableObjectInspector 
+implements VoidObjectInspector{
+
+  WritableVoidObjectInspector() {
+    super(PrimitiveObjectInspectorUtils.voidTypeEntry);
+  }
+  
+  @Override
+  public Object copyObject(Object o) {
+    return o;
+  }
+
+  @Override
+  public Object getPrimitiveJavaObject(Object o) {
+    throw new RuntimeException("Internal error: cannot create Void object.");
+  }
+}

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/thrift/TBinarySortableProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/thrift/TBinarySortableProtocol.java?rev=764994&r1=764993&r2=764994&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/thrift/TBinarySortableProtocol.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/thrift/TBinarySortableProtocol.java Tue Apr 14 22:54:39 2009
@@ -30,6 +30,7 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.io.Text;
 
 import java.util.Properties;
 
@@ -63,7 +64,8 @@
  * This is not thrift compliant in that it doesn't write out field ids
  * so things cannot actually be versioned.
  */
-public class TBinarySortableProtocol extends TProtocol implements ConfigurableTProtocol, WriteNullsProtocol {
+public class TBinarySortableProtocol extends TProtocol implements ConfigurableTProtocol,
+    WriteNullsProtocol, WriteTextProtocol {
 
   final static Log LOG = LogFactory.getLog(TBinarySortableProtocol.class.getName());
   
@@ -300,32 +302,7 @@
     } catch (UnsupportedEncodingException uex) {
       throw new TException("JVM DOES NOT SUPPORT UTF-8: " + uex.getMessage());
     }
-    writeRawBytes(nonNullByte, 0, 1);
-    int begin = 0;
-    int i = 0;
-    for (; i < dat.length; i++) {
-      if (dat[i] == 0 || dat[i] == 1) {
-        // Write the first part of the array
-        if (i > begin) {
-          writeRawBytes(dat, begin, i-begin);
-        }
-        // Write the escaped byte.
-        if (dat[i] == 0) {
-          writeRawBytes(escapedNull, 0, escapedNull.length);
-        } else {
-          writeRawBytes(escapedOne, 0, escapedOne.length);
-        }
-        // Move the pointer to the next byte, since we have written
-        // out the escaped byte in the block above already.
-        begin = i+1;
-      }
-    }
-    // Write the remaining part of the array
-    if (i > begin) {
-      writeRawBytes(dat, begin, i-begin);
-    }
-    // Write the terminating NULL byte
-    writeRawBytes(nullByte, 0, 1);
+    writeTextBytes(dat, 0, dat.length);
   }
 
   public void writeBinary(byte[] bin) throws TException {
@@ -574,5 +551,39 @@
   public void writeNull() throws TException {
     writeRawBytes(nullByte, 0, 1);
   }
+
+  
+  void writeTextBytes(byte[] bytes, int start, int length) throws TException {
+    writeRawBytes(nonNullByte, 0, 1);
+    int begin = 0;
+    int i = start;
+    for (; i < length; i++) {
+      if (bytes[i] == 0 || bytes[i] == 1) {
+        // Write the first part of the array
+        if (i > begin) {
+          writeRawBytes(bytes, begin, i-begin);
+        }
+        // Write the escaped byte.
+        if (bytes[i] == 0) {
+          writeRawBytes(escapedNull, 0, escapedNull.length);
+        } else {
+          writeRawBytes(escapedOne, 0, escapedOne.length);
+        }
+        // Move the pointer to the next byte, since we have written
+        // out the escaped byte in the block above already.
+        begin = i+1;
+      }
+    }
+    // Write the remaining part of the array
+    if (i > begin) {
+      writeRawBytes(bytes, begin, i-begin);
+    }
+    // Write the terminating NULL byte
+    writeRawBytes(nullByte, 0, 1);
+  }
+  
+  public void writeText(Text text) throws TException {
+    writeTextBytes(text.getBytes(), 0, text.getLength());
+  }
   
 }

Added: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/thrift/WriteTextProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/thrift/WriteTextProtocol.java?rev=764994&view=auto
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/thrift/WriteTextProtocol.java (added)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/thrift/WriteTextProtocol.java Tue Apr 14 22:54:39 2009
@@ -0,0 +1,37 @@
+/**
+ * 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.thrift;
+
+import org.apache.hadoop.io.Text;
+
+import com.facebook.thrift.TException;
+
+/**
+ * An interface for TProtocols that can write out data in hadoop Text objects
+ * (UTF-8 encoded String).  This helps a lot with performance because we don't
+ * need to do unnecessary UTF-8 decoding and encoding loops.    
+ */
+public interface WriteTextProtocol {
+
+  /**
+   * Write Text.
+   */
+  public void writeText(Text text) throws TException;
+
+}

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java?rev=764994&r1=764993&r2=764994&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java Tue Apr 14 22:54:39 2009
@@ -63,10 +63,10 @@
 
   public boolean equals(Object other) {
     if (this == other) return true;
-    if (!(other instanceof TypeInfo)) {
+    if (!(other instanceof ListTypeInfo)) {
       return false;
     }
-    TypeInfo o = (TypeInfo) other;
+    ListTypeInfo o = (ListTypeInfo) other;
     return o.getCategory().equals(getCategory())
         && o.getListElementTypeInfo().equals(getListElementTypeInfo());
   }

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java?rev=764994&r1=764993&r2=764994&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java Tue Apr 14 22:54:39 2009
@@ -79,10 +79,10 @@
 
   public boolean equals(Object other) {
     if (this == other) return true;
-    if (!(other instanceof TypeInfo)) {
+    if (!(other instanceof MapTypeInfo)) {
       return false;
     }
-    TypeInfo o = (TypeInfo) other;
+    MapTypeInfo o = (MapTypeInfo) other;
     return o.getCategory().equals(getCategory())
         && o.getMapKeyTypeInfo().equals(getMapKeyTypeInfo())
         && o.getMapValueTypeInfo().equals(getMapValueTypeInfo());

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java?rev=764994&r1=764993&r2=764994&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java Tue Apr 14 22:54:39 2009
@@ -21,6 +21,8 @@
 import java.io.Serializable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 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.PrimitiveObjectInspectorUtils;
 
 
 /** There are limited number of Primitive Types.
@@ -33,49 +35,62 @@
 
   private static final long serialVersionUID = 1L;
   
-  Class<?> primitiveClass;
+  String typeName;
   
   /** For java serialization use only.
    */
   public PrimitiveTypeInfo() {}
 
-  public String getTypeName() {
-    return ObjectInspectorUtils.getClassShortName(primitiveClass);
+  /** For TypeInfoFactory use only.
+   */
+  PrimitiveTypeInfo(String typeName) {
+    this.typeName = typeName;
   }
   
-  
-  /** For java serialization use only.
+  /**
+   * Returns the category of this TypeInfo.
    */
-  public void setPrimitiveClass(Class<?> primitiveClass) {
-    this.primitiveClass = primitiveClass;
+  @Override  
+  public Category getCategory() {
+    return Category.PRIMITIVE;
+  }
+
+  public PrimitiveCategory getPrimitiveCategory() {
+    return PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(typeName).primitiveCategory;
   }
   
-  /** For TypeInfoFactory use only.
-   */
-  PrimitiveTypeInfo(Class<?> primitiveClass) {
-    this.primitiveClass = primitiveClass;
+  public Class<?> getPrimitiveWritableClass() {
+    return PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(typeName).primitiveWritableClass;
   }
   
-  public Category getCategory() {
-    return Category.PRIMITIVE;
+  public Class<?> getPrimitiveJavaClass() {
+    return PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(typeName).primitiveJavaClass;
+  }
+  
+  
+  // The following 2 methods are for java serialization use only.
+  public void setTypeName(String typeName) {
+    this.typeName = typeName;
   }
 
-  public Class<?> getPrimitiveClass() {
-    return primitiveClass;
+  public String getTypeName() {
+    return typeName;
   }
 
+  /**
+   * Compare if 2 TypeInfos are the same.
+   * We use TypeInfoFactory to cache TypeInfos, so we only 
+   * need to compare the Object pointer.
+   */
   public boolean equals(Object other) {
-    if (this == other) return true;
-    if (!(other instanceof TypeInfo)) {
-      return false;
-    }
-    TypeInfo o = (TypeInfo) other;
-    return o.getCategory().equals(getCategory())
-        && o.getPrimitiveClass().equals(getPrimitiveClass());
+    return this == other;
   }
   
+  /**
+   * Generate the hashCode for this TypeInfo.
+   */
   public int hashCode() {
-    return primitiveClass.hashCode();
+    return typeName.hashCode();
   }
   
 }

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java?rev=764994&r1=764993&r2=764994&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java Tue Apr 14 22:54:39 2009
@@ -22,6 +22,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 
 /** StructTypeInfo represents the TypeInfo of a struct.
@@ -45,7 +46,7 @@
 
   public String getTypeName() {
     StringBuilder sb = new StringBuilder();
-    sb.append("struct<");
+    sb.append(Constants.STRUCT_TYPE_NAME + "<");
     for(int i=0; i<allStructFieldNames.size(); i++) {
       if (i>0) sb.append(",");
       sb.append(allStructFieldNames.get(i));
@@ -105,10 +106,10 @@
   @Override
   public boolean equals(Object other) {
     if (this == other) return true;
-    if (!(other instanceof TypeInfo)) {
+    if (!(other instanceof StructTypeInfo)) {
       return false;
     }
-    TypeInfo o = (TypeInfo) other;
+    StructTypeInfo o = (StructTypeInfo) other;
     return o.getCategory().equals(getCategory())
         && o.getAllStructFieldNames().equals(getAllStructFieldNames())
         && o.getAllStructFieldTypeInfos().equals(getAllStructFieldTypeInfos());

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java?rev=764994&r1=764993&r2=764994&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java Tue Apr 14 22:54:39 2009
@@ -19,11 +19,10 @@
 package org.apache.hadoop.hive.serde2.typeinfo;
 
 import java.io.Serializable;
-import java.util.List;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 
 /**
- * Stores information about a type (DDL).
+ * Stores information about a type.
  * Always use the TypeInfoFactory to create new TypeInfo objects.
  * 
  * We support 4 categories of types:
@@ -35,39 +34,18 @@
 public abstract class TypeInfo implements Serializable {
 
   protected TypeInfo() {}
-  
-  public String getTypeName() {
-    throw new RuntimeException("Unsupported: " + this.getClass() + ".getCategory()");
-  }
-  
-  public Category getCategory() {
-    throw new RuntimeException("Unsupported: " + this.getClass() + ".getCategory()");
-  }
-  
-  public Class<?> getPrimitiveClass() {
-    throw new RuntimeException("Unsupported: " + this.getClass() + ".getPrimitiveClass()");
-  }
-  
-  public TypeInfo getListElementTypeInfo() {
-    throw new RuntimeException("Unsupported: " + this.getClass() + ".getListElementTypeInfo()");
-  }
-  
-  public TypeInfo getMapKeyTypeInfo() {
-    throw new RuntimeException("Unsupported: " + this.getClass() + ".getMapKeyTypeInfo()");
-  }
-  public TypeInfo getMapValueTypeInfo() {
-    throw new RuntimeException("Unsupported: " + this.getClass() + ".getMapValueTypeInfo()");
-  }
-  
-  public List<String> getAllStructFieldNames() {
-    throw new RuntimeException("Unsupported: " + this.getClass() + ".getAllStructFieldNames()");
-  }
-  public List<TypeInfo> getAllStructFieldTypeInfos() {
-    throw new RuntimeException("Unsupported: " + this.getClass() + ".getAllStructFieldTypeInfos()");
-  }
-  public TypeInfo getStructFieldTypeInfo(String field) {
-    throw new RuntimeException("Unsupported: " + this.getClass() + ".getStructFieldTypeInfo()");
-  }
+
+  /**
+   * The Category of this TypeInfo.
+   * Possible values are Primitive, List, Map and Struct, which corresponds
+   * to the 4 sub-classes of TypeInfo. 
+   */
+  public abstract Category getCategory();
+  
+  /**
+   * A String representation of the TypeInfo.
+   */
+  public abstract String getTypeName();
   
   public String toString() {
     return getTypeName();

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java?rev=764994&r1=764993&r2=764994&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java Tue Apr 14 22:54:39 2009
@@ -23,7 +23,9 @@
 import java.util.List;
 import java.util.Stack;
 
+import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.apache.hadoop.io.IntWritable;
 
 
@@ -37,17 +39,44 @@
  */
 public class TypeInfoFactory {
 
-  static HashMap<Class<?>, TypeInfo> cachedPrimitiveTypeInfo = new HashMap<Class<?>, TypeInfo>();
-  public static TypeInfo getPrimitiveTypeInfo(Class<?> primitiveClass) {
-    assert(ObjectInspectorUtils.isPrimitiveClass(primitiveClass));
-    primitiveClass = ObjectInspectorUtils.generalizePrimitive(primitiveClass);
-    TypeInfo result = cachedPrimitiveTypeInfo.get(primitiveClass);
+  static HashMap<String, TypeInfo> cachedPrimitiveTypeInfo = new HashMap<String, TypeInfo>();
+  public static TypeInfo getPrimitiveTypeInfo(String typeName) {
+    if (null == PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(typeName)) {
+      throw new RuntimeException("Cannot getPrimitiveTypeInfo for " + typeName);
+    }
+    TypeInfo result = cachedPrimitiveTypeInfo.get(typeName);
     if (result == null) { 
-      result = new PrimitiveTypeInfo(primitiveClass);
-      cachedPrimitiveTypeInfo.put(primitiveClass, result);
+      result = new PrimitiveTypeInfo(typeName);
+      cachedPrimitiveTypeInfo.put(typeName, result);
     }
     return result;
   }
+
+  public static final TypeInfo voidTypeInfo = getPrimitiveTypeInfo(Constants.VOID_TYPE_NAME);
+  public static final TypeInfo booleanTypeInfo = getPrimitiveTypeInfo(Constants.BOOLEAN_TYPE_NAME);
+  public static final TypeInfo intTypeInfo = getPrimitiveTypeInfo(Constants.INT_TYPE_NAME);
+  public static final TypeInfo longTypeInfo = getPrimitiveTypeInfo(Constants.BIGINT_TYPE_NAME);
+  public static final TypeInfo stringTypeInfo = getPrimitiveTypeInfo(Constants.STRING_TYPE_NAME);
+  public static final TypeInfo floatTypeInfo = getPrimitiveTypeInfo(Constants.FLOAT_TYPE_NAME);
+  public static final TypeInfo doubleTypeInfo = getPrimitiveTypeInfo(Constants.DOUBLE_TYPE_NAME);
+  public static final TypeInfo byteTypeInfo = getPrimitiveTypeInfo(Constants.TINYINT_TYPE_NAME);
+  public static final TypeInfo shortTypeInfo = getPrimitiveTypeInfo(Constants.SMALLINT_TYPE_NAME);
+
+  public static final TypeInfo unknownTypeInfo = getPrimitiveTypeInfo("unknown");
+  public static final TypeInfo unknownMapTypeInfo = getPrimitiveTypeInfo(Constants.MAP_TYPE_NAME);
+  public static final TypeInfo unknownListTypeInfo = getPrimitiveTypeInfo(Constants.LIST_TYPE_NAME);
+  
+  public static TypeInfo getPrimitiveTypeInfoFromPrimitiveWritable(Class<?> clazz) {
+    String typeName = PrimitiveObjectInspectorUtils.getTypeNameFromPrimitiveWritable(clazz);
+    if (typeName == null) {
+      throw new RuntimeException("Internal error: Cannot get typeName for " + clazz);
+    }
+    return getPrimitiveTypeInfo(typeName);
+  }
+
+  public static TypeInfo getPrimitiveTypeInfoFromJavaPrimitive(Class<?> clazz) {
+    return getPrimitiveTypeInfo(PrimitiveObjectInspectorUtils.getTypeNameFromPrimitiveJava(clazz));
+  }
   
   static HashMap<ArrayList<List<?>>, TypeInfo> cachedStructTypeInfo = new HashMap<ArrayList<List<?>>, TypeInfo>();
   public static TypeInfo getStructTypeInfo(List<String> names, List<TypeInfo> typeInfos) {

Modified: hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java?rev=764994&r1=764993&r2=764994&view=diff
==============================================================================
--- hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java (original)
+++ hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java Tue Apr 14 22:54:39 2009
@@ -1,8 +1,10 @@
 package org.apache.hadoop.hive.serde2.typeinfo;
 
+import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
@@ -13,9 +15,35 @@
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry;
 
 public class TypeInfoUtils {
   
+  
+  public static List<TypeInfo> getParameterTypeInfos(Method m) {
+    Class<?>[] parameterTypes = m.getParameterTypes();
+    List<TypeInfo> typeInfos = new ArrayList<TypeInfo>(parameterTypes.length);
+    for (int i=0; i<parameterTypes.length; i++) {
+      if (PrimitiveObjectInspectorUtils.isPrimitiveWritableClass(parameterTypes[i])) {
+        typeInfos.add(TypeInfoFactory.getPrimitiveTypeInfoFromPrimitiveWritable(parameterTypes[i]));
+      } else if (PrimitiveObjectInspectorUtils.isPrimitiveJavaClass(parameterTypes[i])
+          || PrimitiveObjectInspectorUtils.isPrimitiveJavaType(parameterTypes[i])) {
+          typeInfos.add(TypeInfoFactory.getPrimitiveTypeInfoFromJavaPrimitive(parameterTypes[i]));
+      } else if (Map.class.isAssignableFrom(parameterTypes[i])) {
+        typeInfos.add(TypeInfoFactory.unknownMapTypeInfo);
+      } else if (List.class.isAssignableFrom(parameterTypes[i])) {
+        typeInfos.add(TypeInfoFactory.unknownListTypeInfo);
+      } else if (parameterTypes[i].equals(Object.class)){
+        typeInfos.add(TypeInfoFactory.unknownTypeInfo);
+      } else {
+        throw new RuntimeException("Hive does not understand type " + parameterTypes[i] + " from " + m);
+      }
+    }
+    return typeInfos;
+  }
   /**
    * Parse a recursive TypeInfo list String.
    * For example, the following inputs are valid inputs: 
@@ -28,12 +56,13 @@
    */
   private static class TypeInfoParser {
     
-    static final String STRUCT_TYPE_NAME = "struct";
-    
     private static class Token {
       public int position;
       public String text;
       public boolean isAlphaDigit;
+      public String toString() {
+        return "" + position + ":" + text;
+      }
     };
     
     /**
@@ -85,7 +114,7 @@
             iToken ++;
           } else {
             throw new IllegalArgumentException("Error: ',', ':', or ';' expected at position " 
-                + separator.position + " from '" + typeInfoString + "'" );
+                + separator.position + " from '" + typeInfoString + "' " + typeInfoTokens );
           }
         }
       }
@@ -105,8 +134,8 @@
       if (item.equals("type")) {
         if (!Constants.LIST_TYPE_NAME.equals(t.text)
             && !Constants.MAP_TYPE_NAME.equals(t.text)
-            && !STRUCT_TYPE_NAME.equals(t.text)
-            && null == ObjectInspectorUtils.typeNameToClass.get(t.text)
+            && !Constants.STRUCT_TYPE_NAME.equals(t.text)
+            && null == PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(t.text)
             && !t.text.equals(alternative)) {
           throw new IllegalArgumentException("Error: " + item + " expected at the position "
               + t.position + " of '" + typeInfoString + "'" );
@@ -131,9 +160,9 @@
       Token t = expect("type");
   
       // Is this a primitive type?
-      Class<?> clazz = ObjectInspectorUtils.typeNameToClass.get(t.text);
-      if (clazz != null) {
-        return TypeInfoFactory.getPrimitiveTypeInfo(clazz);
+      PrimitiveTypeEntry primitiveType = PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(t.text);
+      if (primitiveType != null && !primitiveType.primitiveCategory.equals(PrimitiveCategory.UNKNOWN)) {
+        return TypeInfoFactory.getPrimitiveTypeInfo(primitiveType.typeName);
       }
       
       // Is this a list type?
@@ -155,7 +184,7 @@
       }
   
       // Is this a struct type?
-      if (STRUCT_TYPE_NAME.equals(t.text)) {
+      if (Constants.STRUCT_TYPE_NAME.equals(t.text)) {
         ArrayList<String> fieldNames = new ArrayList<String>();
         ArrayList<TypeInfo> fieldTypeInfos = new ArrayList<TypeInfo>();
         boolean first = true;
@@ -195,23 +224,27 @@
     if (result == null) {
       switch(typeInfo.getCategory()) {
         case PRIMITIVE: {
-          result = ObjectInspectorFactory.getStandardPrimitiveObjectInspector(typeInfo.getPrimitiveClass());
+          result = PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(
+              ((PrimitiveTypeInfo)typeInfo).getPrimitiveCategory());
           break;
         }
         case LIST: {
-          ObjectInspector elementObjectInspector = getStandardObjectInspectorFromTypeInfo(typeInfo.getListElementTypeInfo());
+          ObjectInspector elementObjectInspector = getStandardObjectInspectorFromTypeInfo(
+              ((ListTypeInfo)typeInfo).getListElementTypeInfo());
           result = ObjectInspectorFactory.getStandardListObjectInspector(elementObjectInspector);
           break;
         }
         case MAP: {
-          ObjectInspector keyObjectInspector = getStandardObjectInspectorFromTypeInfo(typeInfo.getMapKeyTypeInfo());
-          ObjectInspector valueObjectInspector = getStandardObjectInspectorFromTypeInfo(typeInfo.getMapValueTypeInfo());
+          MapTypeInfo mapTypeInfo = (MapTypeInfo)typeInfo;
+          ObjectInspector keyObjectInspector = getStandardObjectInspectorFromTypeInfo(mapTypeInfo.getMapKeyTypeInfo());
+          ObjectInspector valueObjectInspector = getStandardObjectInspectorFromTypeInfo(mapTypeInfo.getMapValueTypeInfo());
           result = ObjectInspectorFactory.getStandardMapObjectInspector(keyObjectInspector, valueObjectInspector);
           break;
         }
         case STRUCT: {
-          List<String> fieldNames = typeInfo.getAllStructFieldNames();
-          List<TypeInfo> fieldTypeInfos = typeInfo.getAllStructFieldTypeInfos();
+          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(getStandardObjectInspectorFromTypeInfo(fieldTypeInfos.get(i)));
@@ -228,6 +261,55 @@
     return result;
   }
 
+
+  
+  static HashMap<TypeInfo, ObjectInspector> cachedStandardJavaObjectInspector = new HashMap<TypeInfo, ObjectInspector>();
+  /**
+   * Returns the standard object inspector that can be used to translate an object of that typeInfo
+   * to a standard object type.  
+   */
+  public static ObjectInspector getStandardJavaObjectInspectorFromTypeInfo(TypeInfo typeInfo) {
+    ObjectInspector result = cachedStandardJavaObjectInspector.get(typeInfo);
+    if (result == null) {
+      switch(typeInfo.getCategory()) {
+        case PRIMITIVE: {
+          // NOTE: we use JavaPrimitiveObjectInspector instead of StandardPrimitiveObjectInspector
+          result = PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(
+              PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(typeInfo.getTypeName()).primitiveCategory);
+          break;
+        }
+        case LIST: {
+          ObjectInspector elementObjectInspector = getStandardJavaObjectInspectorFromTypeInfo(
+              ((ListTypeInfo)typeInfo).getListElementTypeInfo());
+          result = ObjectInspectorFactory.getStandardListObjectInspector(elementObjectInspector);
+          break;
+        }
+        case MAP: {
+          MapTypeInfo mapTypeInfo = (MapTypeInfo)typeInfo;
+          ObjectInspector keyObjectInspector = getStandardJavaObjectInspectorFromTypeInfo(mapTypeInfo.getMapKeyTypeInfo());
+          ObjectInspector valueObjectInspector = getStandardJavaObjectInspectorFromTypeInfo(mapTypeInfo.getMapValueTypeInfo());
+          result = ObjectInspectorFactory.getStandardMapObjectInspector(keyObjectInspector, valueObjectInspector);
+          break;
+        }
+        case STRUCT: {
+          StructTypeInfo strucTypeInfo = (StructTypeInfo)typeInfo;
+          List<String> fieldNames = strucTypeInfo.getAllStructFieldNames();
+          List<TypeInfo> fieldTypeInfos = strucTypeInfo.getAllStructFieldTypeInfos();
+          List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(fieldTypeInfos.size());
+          for(int i=0; i<fieldTypeInfos.size(); i++) {
+            fieldObjectInspectors.add(getStandardJavaObjectInspectorFromTypeInfo(fieldTypeInfos.get(i)));
+          }
+          result = ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, fieldObjectInspectors);
+          break;
+        }
+        default: {
+          result = null;
+        }
+      }
+      cachedStandardJavaObjectInspector.put(typeInfo, result);
+    }
+    return result;
+  }
   
   /**
    * Get the TypeInfo object from the ObjectInspector object by recursively going into the
@@ -245,7 +327,7 @@
     switch (oi.getCategory()) {
       case PRIMITIVE: {
         PrimitiveObjectInspector poi =(PrimitiveObjectInspector)oi;
-        result = TypeInfoFactory.getPrimitiveTypeInfo(poi.getPrimitiveClass());
+        result = TypeInfoFactory.getPrimitiveTypeInfo(poi.getTypeName());
         break;
       }
       case LIST: {
@@ -280,44 +362,6 @@
     return result;
   }
     
-  /**
-   * Return the 
-   */
-  public static String getTypeStringFromTypeInfo(TypeInfo typeInfo) {
-    switch(typeInfo.getCategory()) {
-      case PRIMITIVE: {
-        return ObjectInspectorUtils.getClassShortName(typeInfo.getPrimitiveClass());
-      }
-      case LIST: {
-        String elementType = getTypeStringFromTypeInfo(typeInfo.getListElementTypeInfo());
-        return org.apache.hadoop.hive.serde.Constants.LIST_TYPE_NAME + "<" + elementType + ">";
-      }
-      case MAP: {
-        String keyType = getTypeStringFromTypeInfo(typeInfo.getMapKeyTypeInfo());
-        String valueType = getTypeStringFromTypeInfo(typeInfo.getMapValueTypeInfo());
-        return org.apache.hadoop.hive.serde.Constants.MAP_TYPE_NAME + "<" +
-          keyType + "," + valueType + ">";
-      }
-      case STRUCT: {
-        StringBuilder sb = new StringBuilder();
-        sb.append("struct<");
-        List<String> fieldNames = typeInfo.getAllStructFieldNames();
-        List<TypeInfo> fieldTypeInfos = typeInfo.getAllStructFieldTypeInfos();
-        for (int i = 0; i < fieldNames.size(); i++) {
-          if (i>0) sb.append(",");
-          sb.append(fieldNames.get(i));
-          sb.append(":");
-          sb.append(getTypeStringFromTypeInfo(fieldTypeInfos.get(i)));
-        }        
-        sb.append(">");
-        return sb.toString();
-      }
-      default: {
-        throw new RuntimeException("Unknown type!");
-      }
-    }
-  }
-  
   public static ArrayList<TypeInfo> getTypeInfosFromTypeString(String typeString) {
     TypeInfoParser parser = new TypeInfoParser(typeString);
     return parser.parseTypeInfos();