http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaToTypeInfo.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaToTypeInfo.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaToTypeInfo.java
new file mode 100644
index 0000000..c49e890
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaToTypeInfo.java
@@ -0,0 +1,294 @@
+/*
+ * 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.avro;
+
+import static org.apache.avro.Schema.Type.BOOLEAN;
+import static org.apache.avro.Schema.Type.BYTES;
+import static org.apache.avro.Schema.Type.DOUBLE;
+import static org.apache.avro.Schema.Type.FIXED;
+import static org.apache.avro.Schema.Type.FLOAT;
+import static org.apache.avro.Schema.Type.INT;
+import static org.apache.avro.Schema.Type.LONG;
+import static org.apache.avro.Schema.Type.NULL;
+import static org.apache.avro.Schema.Type.STRING;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Hashtable;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.hive.metastore.ColumnType;
+import org.apache.hadoop.hive.metastore.utils.AvroSchemaUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.ITypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.MetastoreTypeInfoUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * Convert an Avro Schema to a Hive TypeInfo
+ */
+@LimitedPrivate("Hive")
+public abstract class SchemaToTypeInfo {
+  // Conversion of Avro primitive types to Hive primitive types
+  // Avro             Hive
+  // Null
+  // boolean          boolean    check
+  // int              int        check
+  // long             bigint     check
+  // float            double     check
+  // double           double     check
+  // bytes            binary     check
+  // fixed            binary     check
+  // string           string     check
+  //                  tinyint
+  //                  smallint
+
+  // Map of Avro's primitive types to Hives (for those that are supported by 
both)
+  private final Map<Schema.Type, TypeInfo> primitiveTypeToTypeInfo;
+  private final ITypeInfoFactory typeInfoFactory;
+  SchemaToTypeInfo(ITypeInfoFactory typeInfoFactory) {
+    this.typeInfoFactory = typeInfoFactory;
+    primitiveTypeToTypeInfo = initTypeMap();
+  }
+
+  private Map<Schema.Type, TypeInfo> initTypeMap() {
+    Map<Schema.Type, TypeInfo> theMap = new Hashtable<Schema.Type, TypeInfo>();
+    theMap.put(NULL, typeInfoFactory.getPrimitiveTypeInfo("void"));
+    theMap.put(BOOLEAN, typeInfoFactory.getPrimitiveTypeInfo("boolean"));
+    theMap.put(INT, typeInfoFactory.getPrimitiveTypeInfo("int"));
+    theMap.put(LONG, typeInfoFactory.getPrimitiveTypeInfo("bigint"));
+    theMap.put(FLOAT, typeInfoFactory.getPrimitiveTypeInfo("float"));
+    theMap.put(DOUBLE, typeInfoFactory.getPrimitiveTypeInfo("double"));
+    theMap.put(BYTES, typeInfoFactory.getPrimitiveTypeInfo("binary"));
+    theMap.put(FIXED, typeInfoFactory.getPrimitiveTypeInfo("binary"));
+    theMap.put(STRING, typeInfoFactory.getPrimitiveTypeInfo("string"));
+    return Collections.unmodifiableMap(theMap);
+  }
+
+  /**
+   * Generate a list of of TypeInfos from an Avro schema.  This method is
+   * currently public due to some weirdness in deserializing unions, but
+   * will be made private once that is resolved.
+   * @param schema Schema to generate field types for
+   * @return List of TypeInfos, each element of which is a TypeInfo derived
+   *         from the schema.
+   * @throws Exception for problems during conversion.
+   */
+  public List<TypeInfo> generateColumnTypes(Schema schema) throws Exception {
+    return generateColumnTypes (schema, null);
+  }
+
+  /**
+   * Generate a list of of TypeInfos from an Avro schema.  This method is
+   * currently public due to some weirdness in deserializing unions, but
+   * will be made private once that is resolved.
+   * @param schema Schema to generate field types for
+   * @param seenSchemas stores schemas processed in the parsing done so far,
+   *         helping to resolve circular references in the schema
+   * @return List of TypeInfos, each element of which is a TypeInfo derived
+   *         from the schema.
+   * @throws Exception for problems during conversion.
+   */
+  public List<TypeInfo> generateColumnTypes(Schema schema,
+      Set<Schema> seenSchemas) throws Exception {
+    List<Schema.Field> fields = schema.getFields();
+
+    List<TypeInfo> types = new ArrayList<TypeInfo>(fields.size());
+
+    for (Schema.Field field : fields) {
+      types.add(generateTypeInfo(field.schema(), seenSchemas));
+    }
+
+    return types;
+  }
+
+  private InstanceCache<Schema, TypeInfo> typeInfoCache = new 
InstanceCache<Schema, TypeInfo>() {
+                                  @Override
+                                  protected TypeInfo makeInstance(Schema s,
+                                      Set<Schema> seenSchemas)
+                                      throws Exception {
+                                    return generateTypeInfoWorker(s, 
seenSchemas);
+                                  }
+                                };
+  /**
+   * Convert an Avro Schema into an equivalent Hive TypeInfo.
+   * @param schema to record. Must be of record type.
+   * @param seenSchemas stores schemas processed in the parsing done so far,
+   *         helping to resolve circular references in the schema
+   * @return TypeInfo matching the Avro schema
+   * @throws Exception for any problems during conversion.
+   */
+  public TypeInfo generateTypeInfo(Schema schema,
+      Set<Schema> seenSchemas) throws Exception {
+    // For bytes type, it can be mapped to decimal.
+    Schema.Type type = schema.getType();
+    if (type == BYTES && AvroSerDeConstants.DECIMAL_TYPE_NAME
+      
.equalsIgnoreCase(schema.getProp(AvroSerDeConstants.AVRO_PROP_LOGICAL_TYPE))) {
+      int precision = 0;
+      int scale = 0;
+      try {
+        precision = 
schema.getJsonProp(AvroSerDeConstants.AVRO_PROP_PRECISION).getIntValue();
+        scale = 
schema.getJsonProp(AvroSerDeConstants.AVRO_PROP_SCALE).getIntValue();
+      } catch (Exception ex) {
+        throw new Exception("Failed to obtain scale value from file schema: " 
+ schema, ex);
+      }
+
+      try {
+        MetastoreTypeInfoUtils.validateDecimalParameters(precision, scale);
+      } catch (Exception ex) {
+        throw new Exception("Invalid precision or scale for decimal type", ex);
+      }
+
+      return 
typeInfoFactory.getPrimitiveTypeInfo(ColumnType.DECIMAL_TYPE_NAME, precision, 
scale);
+    }
+
+    if (type == STRING && AvroSerDeConstants.CHAR_TYPE_NAME
+        
.equalsIgnoreCase(schema.getProp(AvroSerDeConstants.AVRO_PROP_LOGICAL_TYPE))) {
+      int maxLength = 0;
+      try {
+        maxLength = 
schema.getJsonProp(AvroSerDeConstants.AVRO_PROP_MAX_LENGTH).getValueAsInt();
+      } catch (Exception ex) {
+        throw new Exception("Failed to obtain maxLength value from file 
schema: " + schema, ex);
+      }
+      return typeInfoFactory.getPrimitiveTypeInfo(ColumnType.CHAR_TYPE_NAME, 
maxLength);
+    }
+
+    if (type == STRING && AvroSerDeConstants.VARCHAR_TYPE_NAME
+      
.equalsIgnoreCase(schema.getProp(AvroSerDeConstants.AVRO_PROP_LOGICAL_TYPE))) {
+      int maxLength = 0;
+      try {
+        maxLength = 
schema.getJsonProp(AvroSerDeConstants.AVRO_PROP_MAX_LENGTH).getValueAsInt();
+      } catch (Exception ex) {
+        throw new Exception("Failed to obtain maxLength value from file 
schema: " + schema, ex);
+      }
+      return 
typeInfoFactory.getPrimitiveTypeInfo(ColumnType.VARCHAR_TYPE_NAME, maxLength);
+    }
+
+    if (type == INT &&
+        
AvroSerDeConstants.DATE_TYPE_NAME.equals(schema.getProp(AvroSerDeConstants.AVRO_PROP_LOGICAL_TYPE)))
 {
+      return typeInfoFactory.getPrimitiveTypeInfo(ColumnType.DATE_TYPE_NAME);
+    }
+
+    if (type == LONG &&
+        
AvroSerDeConstants.AVRO_TIMESTAMP_TYPE_NAME.equals(schema.getProp(AvroSerDeConstants.AVRO_PROP_LOGICAL_TYPE)))
 {
+      return 
typeInfoFactory.getPrimitiveTypeInfo(ColumnType.TIMESTAMP_TYPE_NAME);
+    }
+
+    return typeInfoCache.retrieve(schema, seenSchemas);
+  }
+
+  private TypeInfo generateTypeInfoWorker(Schema schema,
+      Set<Schema> seenSchemas) throws Exception {
+    // Avro requires NULLable types to be defined as unions of some type 
TypeInfo
+    // and NULL.  This is annoying and we're going to hide it from the user.
+    if(AvroSchemaUtils.isNullableType(schema)) {
+      return generateTypeInfo(
+          AvroSchemaUtils.getOtherTypeFromNullableType(schema), seenSchemas);
+    }
+
+    Schema.Type type = schema.getType();
+    if(primitiveTypeToTypeInfo.containsKey(type)) {
+      return primitiveTypeToTypeInfo.get(type);
+    }
+
+    switch(type) {
+      case RECORD: return generateRecordTypeInfo(schema, seenSchemas);
+      case MAP:    return generateMapTypeInfo(schema, seenSchemas);
+      case ARRAY:  return generateArrayTypeInfo(schema, seenSchemas);
+      case UNION:  return generateUnionTypeInfo(schema, seenSchemas);
+      case ENUM:   return generateEnumTypeInfo(schema);
+      default:     throw new Exception("Do not yet support: " + schema);
+    }
+  }
+
+  private TypeInfo generateRecordTypeInfo(Schema schema,
+      Set<Schema> seenSchemas) throws Exception {
+    assert schema.getType().equals(Schema.Type.RECORD);
+
+    if (seenSchemas == null) {
+        seenSchemas = Collections.newSetFromMap(new IdentityHashMap<Schema, 
Boolean>());
+    } else if (seenSchemas.contains(schema)) {
+      throw new Exception(
+          "Recursive schemas are not supported. Recursive schema was " + schema
+              .getFullName());
+    }
+    seenSchemas.add(schema);
+
+    List<Schema.Field> fields = schema.getFields();
+    List<String> fieldNames = new ArrayList<String>(fields.size());
+    List<TypeInfo> typeInfos = new ArrayList<TypeInfo>(fields.size());
+
+    for(int i = 0; i < fields.size(); i++) {
+      fieldNames.add(i, fields.get(i).name());
+      typeInfos.add(i, generateTypeInfo(fields.get(i).schema(), seenSchemas));
+    }
+
+    return typeInfoFactory.getStructTypeInfo(fieldNames, typeInfos);
+  }
+
+  /**
+   * Generate a TypeInfo for an Avro Map.  This is made slightly simpler in 
that
+   * Avro only allows maps with strings for keys.
+   */
+  private TypeInfo generateMapTypeInfo(Schema schema,
+      Set<Schema> seenSchemas) throws Exception {
+    assert schema.getType().equals(Schema.Type.MAP);
+    Schema valueType = schema.getValueType();
+    TypeInfo ti = generateTypeInfo(valueType, seenSchemas);
+
+    return typeInfoFactory
+        
.getMapTypeInfo(typeInfoFactory.getPrimitiveTypeInfo(ColumnType.STRING_TYPE_NAME),
 ti);
+  }
+
+  private TypeInfo generateArrayTypeInfo(Schema schema,
+      Set<Schema> seenSchemas) throws Exception {
+    assert schema.getType().equals(Schema.Type.ARRAY);
+    Schema itemsType = schema.getElementType();
+    TypeInfo itemsTypeInfo = generateTypeInfo(itemsType, seenSchemas);
+
+    return typeInfoFactory.getListTypeInfo(itemsTypeInfo);
+  }
+
+  private TypeInfo generateUnionTypeInfo(Schema schema,
+      Set<Schema> seenSchemas) throws Exception {
+    assert schema.getType().equals(Schema.Type.UNION);
+    List<Schema> types = schema.getTypes();
+
+
+    List<TypeInfo> typeInfos = new ArrayList<TypeInfo>(types.size());
+
+    for(Schema type : types) {
+      typeInfos.add(generateTypeInfo(type, seenSchemas));
+    }
+
+    return typeInfoFactory.getUnionTypeInfo(typeInfos);
+  }
+
+  // Hive doesn't have an Enum type, so we're going to treat them as Strings.
+  // During the deserialize/serialize stage we'll check for enumness and
+  // convert as such.
+  private TypeInfo generateEnumTypeInfo(Schema schema) {
+    assert schema.getType().equals(Schema.Type.ENUM);
+
+    return typeInfoFactory.getPrimitiveTypeInfo("string");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java
new file mode 100644
index 0000000..604d37a
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java
@@ -0,0 +1,277 @@
+/*
+ * 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.avro;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.hive.metastore.ColumnType;
+import org.apache.hadoop.hive.metastore.utils.AvroSchemaUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MetastorePrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MetastoreTypeInfoUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.node.JsonNodeFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Convert Hive TypeInfo to an Avro Schema
+ */
+public class TypeInfoToSchema {
+
+  private long recordCounter = 0;
+
+  /**
+   * Converts Hive schema to avro schema
+   *
+   * @param columnNames Names of the hive columns
+   * @param columnTypes Hive Column types
+   * @param namespace   Namespace of Avro schema
+   * @param name        Avro schema name
+   * @param doc         Avro schema doc
+   * @return Avro Schema
+   */
+  public Schema convert(List<String> columnNames, List<TypeInfo> columnTypes,
+                        List<String> columnComments, String namespace, String 
name, String doc) {
+
+    List<Schema.Field> fields = new ArrayList<Schema.Field>();
+    for (int i = 0; i < columnNames.size(); ++i) {
+      final String comment = columnComments.size() > i ? columnComments.get(i) 
: null;
+      final Schema.Field avroField = createAvroField(columnNames.get(i), 
columnTypes.get(i),
+          comment);
+      fields.addAll(getFields(avroField));
+    }
+
+    if (name == null || name.isEmpty()) {
+      name = "baseRecord";
+    }
+
+    Schema avroSchema = Schema.createRecord(name, doc, namespace, false);
+    avroSchema.setFields(fields);
+    return avroSchema;
+  }
+
+  private Schema.Field createAvroField(String name, TypeInfo typeInfo, String 
comment) {
+    return new Schema.Field(name, createAvroSchema(typeInfo), comment, null);
+  }
+
+  private Schema createAvroSchema(TypeInfo typeInfo) {
+    Schema schema = null;
+    switch (typeInfo.getCategory()) {
+    case PRIMITIVE:
+      schema = createAvroPrimitive(typeInfo);
+      break;
+    case LIST:
+      schema = createAvroArray(typeInfo);
+      break;
+    case MAP:
+      schema = createAvroMap(typeInfo);
+      break;
+    case STRUCT:
+      schema = createAvroRecord(typeInfo);
+      break;
+    case UNION:
+      schema = createAvroUnion(typeInfo);
+      break;
+    }
+
+    return wrapInUnionWithNull(schema);
+  }
+
+  private Schema createAvroPrimitive(TypeInfo typeInfo) {
+    Schema schema;
+    String baseTypeName = 
MetastoreTypeInfoUtils.getBaseName(typeInfo.getTypeName());
+    switch (baseTypeName) {
+      case ColumnType.STRING_TYPE_NAME:
+        schema = Schema.create(Schema.Type.STRING);
+        break;
+      case ColumnType.CHAR_TYPE_NAME:
+        schema = AvroSchemaUtils.getSchemaFor("{" +
+            "\"type\":\"" + AvroSerDeConstants.AVRO_STRING_TYPE_NAME + "\"," +
+            "\"logicalType\":\"" + AvroSerDeConstants.CHAR_TYPE_NAME + "\"," +
+            "\"maxLength\":" + ((MetastorePrimitiveTypeInfo) 
typeInfo).getParameters()[0] + "}");
+        break;
+      case ColumnType.VARCHAR_TYPE_NAME:
+        schema = AvroSchemaUtils.getSchemaFor("{" +
+            "\"type\":\"" + AvroSerDeConstants.AVRO_STRING_TYPE_NAME + "\"," +
+            "\"logicalType\":\"" + AvroSerDeConstants.VARCHAR_TYPE_NAME + 
"\"," +
+            "\"maxLength\":" + ((MetastorePrimitiveTypeInfo) 
typeInfo).getParameters()[0] + "}");
+        break;
+      case ColumnType.BINARY_TYPE_NAME:
+        schema = Schema.create(Schema.Type.BYTES);
+        break;
+      case ColumnType.TINYINT_TYPE_NAME:
+        schema = Schema.create(Schema.Type.INT);
+        break;
+      case ColumnType.SMALLINT_TYPE_NAME:
+        schema = Schema.create(Schema.Type.INT);
+        break;
+      case ColumnType.INT_TYPE_NAME:
+        schema = Schema.create(Schema.Type.INT);
+        break;
+      case ColumnType.BIGINT_TYPE_NAME:
+        schema = Schema.create(Schema.Type.LONG);
+        break;
+      case ColumnType.FLOAT_TYPE_NAME:
+        schema = Schema.create(Schema.Type.FLOAT);
+        break;
+      case ColumnType.DOUBLE_TYPE_NAME:
+        schema = Schema.create(Schema.Type.DOUBLE);
+        break;
+      case ColumnType.BOOLEAN_TYPE_NAME:
+        schema = Schema.create(Schema.Type.BOOLEAN);
+        break;
+      case ColumnType.DECIMAL_TYPE_NAME:
+        String precision = String.valueOf(((MetastorePrimitiveTypeInfo) 
typeInfo).getParameters()[0]);
+        String scale = String.valueOf(((MetastorePrimitiveTypeInfo) 
typeInfo).getParameters()[1]);
+        schema = AvroSchemaUtils.getSchemaFor("{" +
+            "\"type\":\"bytes\"," +
+            "\"logicalType\":\"decimal\"," +
+            "\"precision\":" + precision + "," +
+            "\"scale\":" + scale + "}");
+        break;
+      case ColumnType.DATE_TYPE_NAME:
+        schema = AvroSchemaUtils.getSchemaFor("{" +
+            "\"type\":\"" + AvroSerDeConstants.AVRO_INT_TYPE_NAME + "\"," +
+            "\"logicalType\":\"" + AvroSerDeConstants.DATE_TYPE_NAME + "\"}");
+        break;
+      case ColumnType.TIMESTAMP_TYPE_NAME:
+        schema = AvroSchemaUtils.getSchemaFor("{" +
+          "\"type\":\"" + AvroSerDeConstants.AVRO_LONG_TYPE_NAME + "\"," +
+          "\"logicalType\":\"" + AvroSerDeConstants.AVRO_TIMESTAMP_TYPE_NAME + 
"\"}");
+        break;
+      case ColumnType.VOID_TYPE_NAME:
+        schema = Schema.create(Schema.Type.NULL);
+        break;
+      default:
+        throw new UnsupportedOperationException(typeInfo + " is not 
supported.");
+    }
+    return schema;
+  }
+
+  private Schema createAvroUnion(TypeInfo typeInfo) {
+    List<Schema> childSchemas = new ArrayList<Schema>();
+    for (TypeInfo childTypeInfo : ((UnionTypeInfo) 
typeInfo).getAllUnionObjectTypeInfos()) {
+      final Schema childSchema = createAvroSchema(childTypeInfo);
+      if (childSchema.getType() == Schema.Type.UNION) {
+        childSchemas.addAll(childSchema.getTypes());
+      } else {
+        childSchemas.add(childSchema);
+      }
+    }
+    return Schema.createUnion(removeDuplicateNullSchemas(childSchemas));
+  }
+
+  private Schema createAvroRecord(TypeInfo typeInfo) {
+    List<Schema.Field> childFields = new ArrayList<Schema.Field>();
+    final List<String> allStructFieldNames = ((StructTypeInfo) 
typeInfo).getAllStructFieldNames();
+    final List<TypeInfo> allStructFieldTypeInfos =
+        ((StructTypeInfo) typeInfo).getAllStructFieldTypeInfos();
+    if (allStructFieldNames.size() != allStructFieldTypeInfos.size()) {
+      throw new IllegalArgumentException("Failed to generate avro schema from 
hive schema. " +
+          "name and column type differs. names = " + allStructFieldNames + ", 
types = " +
+          allStructFieldTypeInfos);
+    }
+
+    for (int i = 0; i < allStructFieldNames.size(); ++i) {
+      final TypeInfo childTypeInfo = allStructFieldTypeInfos.get(i);
+      final Schema.Field grandChildSchemaField = 
createAvroField(allStructFieldNames.get(i),
+          childTypeInfo, childTypeInfo.toString());
+      final List<Schema.Field> grandChildFields = 
getFields(grandChildSchemaField);
+      childFields.addAll(grandChildFields);
+    }
+
+    Schema recordSchema = Schema.createRecord("record_" + recordCounter, 
typeInfo.toString(),
+        null, false);
+    ++recordCounter;
+    recordSchema.setFields(childFields);
+    return recordSchema;
+  }
+
+  private Schema createAvroMap(TypeInfo typeInfo) {
+    TypeInfo keyTypeInfo = ((MapTypeInfo) typeInfo).getMapKeyTypeInfo();
+    if (!ColumnType.STRING_TYPE_NAME.equals(keyTypeInfo.getTypeName())) {
+      throw new UnsupportedOperationException("Key of Map can only be a 
String");
+    }
+
+    TypeInfo valueTypeInfo = ((MapTypeInfo) typeInfo).getMapValueTypeInfo();
+    Schema valueSchema = createAvroSchema(valueTypeInfo);
+
+    return Schema.createMap(valueSchema);
+  }
+
+  private Schema createAvroArray(TypeInfo typeInfo) {
+    ListTypeInfo listTypeInfo = (ListTypeInfo) typeInfo;
+    Schema listSchema = 
createAvroSchema(listTypeInfo.getListElementTypeInfo());
+    return Schema.createArray(listSchema);
+  }
+
+  private List<Schema.Field> getFields(Schema.Field schemaField) {
+    List<Schema.Field> fields = new ArrayList<Schema.Field>();
+
+    JsonNode nullDefault = JsonNodeFactory.instance.nullNode();
+    if (schemaField.schema().getType() == Schema.Type.RECORD) {
+      for (Schema.Field field : schemaField.schema().getFields()) {
+        fields.add(new Schema.Field(field.name(), field.schema(), field.doc(), 
nullDefault));
+      }
+    } else {
+      fields.add(new Schema.Field(schemaField.name(), schemaField.schema(), 
schemaField.doc(),
+          nullDefault));
+    }
+
+    return fields;
+  }
+
+  private Schema wrapInUnionWithNull(Schema schema) {
+    Schema wrappedSchema = schema;
+    switch (schema.getType()) {
+      case NULL:
+        break;
+      case UNION:
+        List<Schema> existingSchemas = 
removeDuplicateNullSchemas(schema.getTypes());
+        wrappedSchema = Schema.createUnion(existingSchemas);
+        break;
+      default:
+        wrappedSchema = 
Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), schema));
+    }
+
+    return wrappedSchema;
+  }
+
+  private List<Schema> removeDuplicateNullSchemas(List<Schema> childSchemas) {
+    List<Schema> prunedSchemas = new ArrayList<Schema>();
+    boolean isNullPresent = false;
+    for (Schema schema : childSchemas) {
+      if (schema.getType() == Schema.Type.NULL) {
+        isNullPresent = true;
+      } else {
+        prunedSchemas.add(schema);
+      }
+    }
+    if (isNullPresent) {
+      prunedSchemas.add(0, Schema.create(Schema.Type.NULL));
+    }
+
+    return prunedSchemas;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/ITypeInfoFactory.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/ITypeInfoFactory.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/ITypeInfoFactory.java
new file mode 100644
index 0000000..24edf70
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/ITypeInfoFactory.java
@@ -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.typeinfo;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+
+import java.util.List;
+
+@LimitedPrivate("Hive")
+public interface ITypeInfoFactory {
+  /**
+   * Get or create a Primitive TypeInfo object of name typeName and parameters 
provided by
+   * paramaters. Eg. a primitive typeInfo of char(10) can be represented a 
typename --> char
+   * and 10 as the parameter. Similarly, decimal(10,2) has typename decimal 
and 10,2 as
+   * parameters
+   *
+   * @param typeName   name of the type
+   * @param parameters optional parameters in case of parameterized primitive 
types
+   * @return TypeInfo representing the primitive typeInfo
+   */
+  MetastorePrimitiveTypeInfo getPrimitiveTypeInfo(String typeName, Object... 
parameters);
+
+  /**
+   * Get or create a Map type TypeInfo
+   *
+   * @param keyTypeInfo   TypeInfo for the key
+   * @param valueTypeInfo TypeInfo for the value
+   * @return MapTypeInfo
+   */
+  MapTypeInfo getMapTypeInfo(TypeInfo keyTypeInfo, TypeInfo valueTypeInfo);
+
+  /**
+   * Get or create a List type TypeInfo
+   *
+   * @param listElementTypeInfo TypeInfo of the list elements
+   * @return ListTypeInfo
+   */
+  ListTypeInfo getListTypeInfo(TypeInfo listElementTypeInfo);
+
+  /**
+   * Get or create a UnionTypeInfo
+   *
+   * @param typeInfos child TypeInfos for the UnionTypeInfo
+   * @return UnionTypeInfo
+   */
+  UnionTypeInfo getUnionTypeInfo(List<TypeInfo> typeInfos);
+
+  /**
+   * Get or create a StructTypeInfo
+   *
+   * @param names     names of the fields in the struct typeInfo
+   * @param typeInfos TypeInfos for each fields
+   * @return StructTypeInfo
+   */
+  StructTypeInfo getStructTypeInfo(List<String> names, List<TypeInfo> 
typeInfos);
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java
new file mode 100644
index 0000000..e9335bf
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java
@@ -0,0 +1,94 @@
+/*
+ * 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.typeinfo;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.ColumnType;
+
+import java.io.Serializable;
+
+
+/**
+ * A List Type has homogeneous elements. All elements of the List has the same
+ * TypeInfo which is returned by getListElementTypeInfo.
+ *
+ * Always use the TypeInfoFactory to create new TypeInfo objects, instead of
+ * directly creating an instance of this class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public final class ListTypeInfo extends TypeInfo implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+  private TypeInfo listElementTypeInfo;
+
+  /**
+   * For java serialization use only.
+   */
+  public ListTypeInfo() {
+  }
+
+  @Override
+  public String getTypeName() {
+    return ColumnType.LIST_TYPE_NAME + "<"
+        + listElementTypeInfo.getTypeName() + ">";
+  }
+
+  /**
+   * For java serialization use only.
+   */
+  public void setListElementTypeInfo(TypeInfo listElementTypeInfo) {
+    this.listElementTypeInfo = listElementTypeInfo;
+  }
+
+  /**
+   * For TypeInfoFactory use only.
+   */
+  ListTypeInfo(TypeInfo elementTypeInfo) {
+    listElementTypeInfo = elementTypeInfo;
+  }
+
+  @Override
+  public MetastoreTypeCategory getCategory() {
+    return MetastoreTypeCategory.LIST;
+  }
+
+  public TypeInfo getListElementTypeInfo() {
+    return listElementTypeInfo;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (!(other instanceof ListTypeInfo)) {
+      return false;
+    }
+    return getListElementTypeInfo().equals(
+        ((ListTypeInfo) other).getListElementTypeInfo());
+  }
+
+  @Override
+  public int hashCode() {
+    return listElementTypeInfo.hashCode();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java
new file mode 100644
index 0000000..f156bb5
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java
@@ -0,0 +1,110 @@
+/*
+ * 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.typeinfo;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.ColumnType;
+
+import java.io.Serializable;
+
+
+/**
+ * A Map Type has homogeneous keys and homogeneous values. All keys of the Map
+ * have the same TypeInfo, which is returned by getMapKeyTypeInfo(); and all
+ * values of the Map has the same TypeInfo, which is returned by
+ * getMapValueTypeInfo().
+ *
+ * Always use the TypeInfoFactory to create new TypeInfo objects, instead of
+ * directly creating an instance of this class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public final class MapTypeInfo extends TypeInfo implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  private TypeInfo mapKeyTypeInfo;
+  private TypeInfo mapValueTypeInfo;
+
+  /**
+   * For java serialization use only.
+   */
+  public MapTypeInfo() {
+  }
+
+  @Override
+  public String getTypeName() {
+    return ColumnType.MAP_TYPE_NAME + "<"
+        + mapKeyTypeInfo.getTypeName() + "," + mapValueTypeInfo.getTypeName()
+        + ">";
+  }
+
+  /**
+   * For java serialization use only.
+   */
+  public void setMapKeyTypeInfo(TypeInfo mapKeyTypeInfo) {
+    this.mapKeyTypeInfo = mapKeyTypeInfo;
+  }
+
+  /**
+   * For java serialization use only.
+   */
+  public void setMapValueTypeInfo(TypeInfo mapValueTypeInfo) {
+    this.mapValueTypeInfo = mapValueTypeInfo;
+  }
+
+  // For TypeInfoFactory use only
+  MapTypeInfo(TypeInfo keyTypeInfo, TypeInfo valueTypeInfo) {
+    mapKeyTypeInfo = keyTypeInfo;
+    mapValueTypeInfo = valueTypeInfo;
+  }
+
+  @Override
+  public MetastoreTypeCategory getCategory() {
+    return MetastoreTypeCategory.MAP;
+  }
+
+  public TypeInfo getMapKeyTypeInfo() {
+    return mapKeyTypeInfo;
+  }
+
+  public TypeInfo getMapValueTypeInfo() {
+    return mapValueTypeInfo;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (!(other instanceof MapTypeInfo)) {
+      return false;
+    }
+    MapTypeInfo o = (MapTypeInfo) other;
+    return o.getMapKeyTypeInfo().equals(getMapKeyTypeInfo())
+        && o.getMapValueTypeInfo().equals(getMapValueTypeInfo());
+  }
+
+  @Override
+  public int hashCode() {
+    return mapKeyTypeInfo.hashCode() ^ mapValueTypeInfo.hashCode();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastorePrimitiveTypeInfo.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastorePrimitiveTypeInfo.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastorePrimitiveTypeInfo.java
new file mode 100644
index 0000000..c2ce312
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastorePrimitiveTypeInfo.java
@@ -0,0 +1,91 @@
+/*
+ * 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.typeinfo;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+
+import java.io.Serializable;
+
+/**
+ * This class represents a PrimitiveTypeInfo. Hive extends this class to 
create PrimitiveTypeInfo
+ */
+@LimitedPrivate("Hive")
+public class MetastorePrimitiveTypeInfo extends TypeInfo implements 
Serializable {
+  // Base name (varchar vs fully qualified name such as varchar(200)).
+  protected String typeName;
+
+  public MetastorePrimitiveTypeInfo() {
+  }
+
+  public MetastorePrimitiveTypeInfo(String typeName) {
+    this.typeName = typeName;
+  }
+
+  // The following 2 methods are for java serialization use only.
+  public void setTypeName(String typeName) {
+    this.typeName = typeName;
+  }
+
+  @Override
+  public String getTypeName() {
+    return typeName;
+  }
+
+  @Override
+  public MetastoreTypeCategory getCategory() {
+    return MetastoreTypeCategory.PRIMITIVE;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (other == null || getClass() != other.getClass()) {
+      return false;
+    }
+
+    MetastorePrimitiveTypeInfo pti = (MetastorePrimitiveTypeInfo) other;
+
+    return this.typeName.equals(pti.typeName);
+  }
+
+  /**
+   * Generate the hashCode for this TypeInfo.
+   */
+  @Override
+  public int hashCode() {
+    return typeName.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return typeName;
+  }
+
+  private static final Object[] EMPTY_OBJECT_ARRAY = new Object[0];
+
+  /**
+   * parameterized TypeInfos should override this to return array of parameters
+   * @return
+   */
+  public Object[] getParameters() {
+    //default is no parameters
+    return EMPTY_OBJECT_ARRAY;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeCategory.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeCategory.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeCategory.java
new file mode 100644
index 0000000..03b764b
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeCategory.java
@@ -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.typeinfo;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+
+/**
+ * This enum maps to the ObjectInspector.Category enum from Hive source code. 
This was added
+ * as part of metastore separation from Hive and is used by the storage schema 
readers in metastore.
+ * It is very important that the enum values here should match enum values of 
ObjectInspector.Category
+ * enum in hive source code.
+ */
+@LimitedPrivate("Hive")
+public enum MetastoreTypeCategory {
+  PRIMITIVE,
+  LIST,
+  STRUCT,
+  MAP,
+  UNION
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeInfoFactory.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeInfoFactory.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeInfoFactory.java
new file mode 100644
index 0000000..de10641
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeInfoFactory.java
@@ -0,0 +1,128 @@
+/*
+ * 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.typeinfo;
+
+import org.apache.hadoop.hive.metastore.ColumnType;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class MetastoreTypeInfoFactory implements ITypeInfoFactory {
+
+  private static final MetastoreTypeInfoFactory instance = new 
MetastoreTypeInfoFactory();
+
+  public static final MetastoreTypeInfoFactory getInstance() {
+    return instance;
+  }
+  private static ConcurrentHashMap<String, MetastorePrimitiveTypeInfo> 
cachedPrimitiveTypeInfo =
+      new ConcurrentHashMap<>();
+
+  @Override
+  public MetastorePrimitiveTypeInfo getPrimitiveTypeInfo(String typeName, 
Object... parameters) {
+    String qualifiedTypeName = MetastoreTypeInfoUtils
+        .getQualifiedPrimitiveTypeName(typeName, parameters);
+    MetastorePrimitiveTypeInfo result = 
cachedPrimitiveTypeInfo.get(qualifiedTypeName);
+    if (result != null) {
+      return result;
+    }
+
+    if (ColumnType.CHAR_TYPE_NAME.equals(typeName) || 
ColumnType.VARCHAR_TYPE_NAME
+        .equals(typeName)) {
+      MetastoreTypeInfoUtils.validateCharVarCharParameters((int) 
parameters[0]);
+    } else if (ColumnType.DECIMAL_TYPE_NAME.equals(typeName)) {
+      MetastoreTypeInfoUtils.validateDecimalParameters((int) parameters[0], 
(int) parameters[1]);
+    }
+    // Not found in the cache. Must be parameterized types. Create it.
+    result = new MetastorePrimitiveTypeInfo(qualifiedTypeName);
+
+    MetastorePrimitiveTypeInfo prev = 
cachedPrimitiveTypeInfo.putIfAbsent(qualifiedTypeName, result);
+    if (prev != null) {
+      result = prev;
+    }
+    return result;
+  }
+
+  private static ConcurrentHashMap<ArrayList<TypeInfo>, MapTypeInfo> 
cachedMapTypeInfo =
+      new ConcurrentHashMap<>();
+  @Override
+  public MapTypeInfo getMapTypeInfo(TypeInfo keyTypeInfo,
+      TypeInfo valueTypeInfo) {
+    ArrayList<TypeInfo> signature = new ArrayList<TypeInfo>(2);
+    signature.add(keyTypeInfo);
+    signature.add(valueTypeInfo);
+    MapTypeInfo result = cachedMapTypeInfo.get(signature);
+    if (result == null) {
+      result = new MapTypeInfo(keyTypeInfo, valueTypeInfo);
+      MapTypeInfo prev = cachedMapTypeInfo.putIfAbsent(signature, result);
+      if (prev != null) {
+        result = prev;
+      }
+    }
+    return result;
+  }
+
+  private static ConcurrentHashMap<TypeInfo, ListTypeInfo> cachedListTypeInfo 
= new ConcurrentHashMap<>();
+
+  @Override
+  public ListTypeInfo getListTypeInfo(TypeInfo listElementTypeInfo) {
+    ListTypeInfo result = cachedListTypeInfo.get(listElementTypeInfo);
+    if (result == null) {
+      result = new ListTypeInfo(listElementTypeInfo);
+      ListTypeInfo prev = cachedListTypeInfo.putIfAbsent(listElementTypeInfo, 
result);
+      if (prev != null) {
+        result = prev;
+      }
+    }
+    return result;
+  }
+
+  private static ConcurrentHashMap<List<?>, UnionTypeInfo> cachedUnionTypeInfo 
=
+      new ConcurrentHashMap<>();
+
+  @Override
+  public UnionTypeInfo getUnionTypeInfo(List<TypeInfo> typeInfos) {
+    UnionTypeInfo result = cachedUnionTypeInfo.get(typeInfos);
+    if (result == null) {
+      result = new UnionTypeInfo(typeInfos);
+      UnionTypeInfo prev = cachedUnionTypeInfo.putIfAbsent(typeInfos, result);
+      if (prev != null) {
+        result = prev;
+      }
+    }
+    return result;
+  }
+  static ConcurrentHashMap<ArrayList<List<?>>, StructTypeInfo> 
cachedStructTypeInfo =
+      new ConcurrentHashMap<>();
+  @Override
+  public StructTypeInfo getStructTypeInfo(List<String> names,
+      List<TypeInfo> typeInfos) {
+      ArrayList<List<?>> signature = new ArrayList<List<?>>(2);
+      signature.add(names);
+      signature.add(typeInfos);
+    StructTypeInfo result = cachedStructTypeInfo.get(signature);
+      if (result == null) {
+        result = new StructTypeInfo(names, typeInfos);
+        StructTypeInfo prev = cachedStructTypeInfo.putIfAbsent(signature, 
result);
+        if (prev != null) {
+          result = prev;
+        }
+      }
+      return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeInfoUtils.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeInfoUtils.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeInfoUtils.java
new file mode 100644
index 0000000..780dc50
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/MetastoreTypeInfoUtils.java
@@ -0,0 +1,59 @@
+package org.apache.hadoop.hive.serde2.typeinfo;
+
+public class MetastoreTypeInfoUtils {
+  private MetastoreTypeInfoUtils() {
+  }
+
+  /**
+   * Metastore is not supposed to enforce type ranges. The type range checks 
should be left
+   * to the implementation engines. This method does a very lenient check 
which is obvious
+   * and makes sense for overall sanity of decimal types
+   * @param precision decimal precision value
+   * @param scale decimal scale value
+   */
+  public static void validateDecimalParameters(int precision, int scale) {
+    if (precision < 0) {
+      throw new IllegalArgumentException("Precision cannot be negative");
+    }
+    if (scale < 0) {
+      throw new IllegalArgumentException("Scale cannot be negative");
+    }
+  }
+
+  /**
+   * Metastore is not supposed to enforce type ranges. The type range checks 
should be left
+   * to the implementation engines. This method does a very lenient check 
which is obvious
+   * and makes sense for overall sanity of char types
+   * @param length
+   */
+  public static void validateCharVarCharParameters(int length) {
+    if (length < 0) {
+      throw new IllegalArgumentException("Length cannot be negative");
+    }
+  }
+
+  static String getQualifiedPrimitiveTypeName(String type, Object... 
parameters) {
+    StringBuilder sb = new StringBuilder(type);
+    if (parameters == null || parameters.length == 0) {
+      return sb.toString();
+    }
+    sb.append('(');
+    for (int i = 0; i < parameters.length; i++) {
+      sb.append(parameters[i]);
+      if (i != (parameters.length - 1)) {
+        sb.append(',');
+      }
+    }
+    sb.append(')');
+    return sb.toString();
+  }
+
+  public static String getBaseName(String typeName) {
+    int idx = typeName.indexOf('(');
+    if (idx == -1) {
+      return typeName;
+    } else {
+      return typeName.substring(0, idx);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java
new file mode 100644
index 0000000..824c19b
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java
@@ -0,0 +1,150 @@
+/*
+ * 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.typeinfo;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.ColumnType;
+
+/**
+ * StructTypeInfo represents the TypeInfo of a struct. A struct contains one or
+ * more fields each of which has a unique name and its own TypeInfo. Different
+ * fields can have the same or different TypeInfo.
+ *
+ * Always use the TypeInfoFactory to create new TypeInfo objects, instead of
+ * directly creating an instance of this class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public final class StructTypeInfo extends TypeInfo implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  private ArrayList<String> allStructFieldNames;
+  private ArrayList<TypeInfo> allStructFieldTypeInfos;
+
+  /**
+   * For java serialization use only.
+   */
+  public StructTypeInfo() {
+  }
+
+  @Override
+  public String getTypeName() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(ColumnType.STRUCT_TYPE_NAME + "<");
+    for (int i = 0; i < allStructFieldNames.size(); i++) {
+      if (i > 0) {
+        sb.append(",");
+      }
+      sb.append(allStructFieldNames.get(i));
+      sb.append(":");
+      sb.append(allStructFieldTypeInfos.get(i).getTypeName());
+    }
+    sb.append(">");
+    return sb.toString();
+  }
+
+  /**
+   * For java serialization use only.
+   */
+  public void setAllStructFieldNames(ArrayList<String> allStructFieldNames) {
+    this.allStructFieldNames = allStructFieldNames;
+  }
+
+  /**
+   * For java serialization use only.
+   */
+  public void setAllStructFieldTypeInfos(
+      ArrayList<TypeInfo> allStructFieldTypeInfos) {
+    this.allStructFieldTypeInfos = allStructFieldTypeInfos;
+  }
+
+  /**
+   * For TypeInfoFactory use only.
+   */
+  StructTypeInfo(List<String> names, List<TypeInfo> typeInfos) {
+    allStructFieldNames = new ArrayList<String>(names);
+    allStructFieldTypeInfos = new ArrayList<TypeInfo>(typeInfos);
+  }
+
+  @Override
+  public MetastoreTypeCategory getCategory() {
+    return MetastoreTypeCategory.STRUCT;
+  }
+
+  public ArrayList<String> getAllStructFieldNames() {
+    return allStructFieldNames;
+  }
+
+  public ArrayList<TypeInfo> getAllStructFieldTypeInfos() {
+    return allStructFieldTypeInfos;
+  }
+
+  public TypeInfo getStructFieldTypeInfo(String field) {
+    String fieldLowerCase = field.toLowerCase();
+    for (int i = 0; i < allStructFieldNames.size(); i++) {
+      if (fieldLowerCase.equalsIgnoreCase(allStructFieldNames.get(i))) {
+        return allStructFieldTypeInfos.get(i);
+      }
+    }
+    throw new RuntimeException("cannot find field " + field
+        + "(lowercase form: " + fieldLowerCase + ") in " + 
allStructFieldNames);
+    // return null;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (!(other instanceof StructTypeInfo)) {
+      return false;
+    }
+    StructTypeInfo o = (StructTypeInfo) other;
+    Iterator<String> namesIterator = getAllStructFieldNames().iterator();
+    Iterator<String> otherNamesIterator = 
o.getAllStructFieldNames().iterator();
+
+    // Compare the field names using ignore-case semantics
+    while (namesIterator.hasNext() && otherNamesIterator.hasNext()) {
+      if (!namesIterator.next().equalsIgnoreCase(otherNamesIterator.next())) {
+        return false;
+      }
+    }
+
+    // Different number of field names
+    if (namesIterator.hasNext() || otherNamesIterator.hasNext()) {
+      return false;
+    }
+
+    // Compare the field types
+    return o.getAllStructFieldTypeInfos().equals(getAllStructFieldTypeInfos());
+  }
+
+  @Override
+  public int hashCode() {
+    return allStructFieldNames.hashCode() ^ allStructFieldTypeInfos.hashCode();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java
new file mode 100644
index 0000000..5bfb207
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java
@@ -0,0 +1,82 @@
+/*
+ * 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.typeinfo;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Stores information about a type. Always use the TypeInfoFactory to create 
new
+ * TypeInfo objects.
+ *
+ * We support 8 categories of types:
+ * 1. Primitive objects (String, Number, etc)
+ * 2. List objects (a list of objects of a single type)
+ * 3. Map objects (a map from objects of one type to objects of another type)
+ * 4. Struct objects (a list of fields with names and their own types)
+ * 5. Union objects
+ * 6. Decimal objects
+ * 7. Char objects
+ * 8. Varchar objects
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class TypeInfo implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+  protected TypeInfo() {
+  }
+
+  /**
+   * A String representation of the TypeInfo.
+   */
+  public abstract String getTypeName();
+
+  /**
+   *
+   * @return
+   */
+  public abstract MetastoreTypeCategory getCategory();
+
+  /**
+   * String representing the qualified type name.
+   * Qualified types should override this method.
+   * @return
+   */
+  public String getQualifiedName() {
+    return getTypeName();
+  }
+
+  @Override
+  public String toString() {
+    return getTypeName();
+  }
+
+  @Override
+  public abstract boolean equals(Object o);
+
+  @Override
+  public abstract int hashCode();
+
+  public boolean accept(TypeInfo other) {
+    return this.equals(other);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoParser.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoParser.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoParser.java
new file mode 100644
index 0000000..7bdad93
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoParser.java
@@ -0,0 +1,343 @@
+/*
+ * 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.typeinfo;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.metastore.ColumnType;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Parse a recursive TypeInfo list String. For example, the following inputs
+ * are valid inputs:
+ * 
"int,string,map<string,int>,list<map<int,list<string>>>,list<struct<a:int,b:string>>"
+ * The separators between TypeInfos can be ",", ":", or ";".
+ *
+ * In order to use this class: TypeInfoParser parser = new
+ * TypeInfoParser("int,string"); ArrayList<TypeInfo> typeInfos =
+ * parser.parseTypeInfos();
+ */
+@LimitedPrivate("Hive")
+public class TypeInfoParser {
+
+  private static class Token {
+    public int position;
+    public String text;
+    public boolean isType;
+
+    @Override
+    public String toString() {
+      return "" + position + ":" + text;
+    }
+  };
+
+  private static boolean isTypeChar(char c) {
+    return Character.isLetterOrDigit(c) || c == '_' || c == '.' || c == ' ' || 
c == '$';
+  }
+
+  /**
+   * Tokenize the typeInfoString. The rule is simple: all consecutive
+   * alphadigits and '_', '.' are in one token, and all other characters are
+   * one character per token.
+   *
+   * tokenize("map<int,string>") should return
+   * ["map","<","int",",","string",">"]
+   *
+   * Note that we add '$' in new Calcite return path. As '$' will not appear
+   * in any type in Hive, it is safe to do so.
+   */
+  private static ArrayList<Token> tokenize(String typeInfoString) {
+    ArrayList<Token> tokens = new ArrayList<Token>(0);
+    int begin = 0;
+    int end = 1;
+    while (end <= typeInfoString.length()) {
+      // last character ends a token?
+      // if there are quotes, all the text between the quotes
+      // is considered a single token (this can happen for
+      // timestamp with local time-zone)
+      if (begin > 0 &&
+          typeInfoString.charAt(begin - 1) == '(' &&
+          typeInfoString.charAt(begin) == '\'') {
+        // Ignore starting quote
+        begin++;
+        do {
+          end++;
+        } while (typeInfoString.charAt(end) != '\'');
+      } else if (typeInfoString.charAt(begin) == '\'' &&
+          typeInfoString.charAt(begin + 1) == ')') {
+        // Ignore closing quote
+        begin++;
+        end++;
+      }
+      if (end == typeInfoString.length()
+          || !isTypeChar(typeInfoString.charAt(end - 1))
+          || !isTypeChar(typeInfoString.charAt(end))) {
+        Token t = new Token();
+        t.position = begin;
+        t.text = typeInfoString.substring(begin, end);
+        t.isType = isTypeChar(typeInfoString.charAt(begin));
+        tokens.add(t);
+        begin = end;
+      }
+      end++;
+    }
+    return tokens;
+  }
+
+  public TypeInfoParser(String typeInfoString, ITypeInfoFactory 
typeInfoFactory) {
+    this.typeInfoString = typeInfoString;
+    this.typeInfoFactory = typeInfoFactory;
+    typeInfoTokens = tokenize(typeInfoString);
+  }
+
+  private final String typeInfoString;
+  private final ArrayList<Token> typeInfoTokens;
+  private final ITypeInfoFactory typeInfoFactory;
+  private ArrayList<TypeInfo> typeInfos;
+  private int iToken;
+
+  public ArrayList<TypeInfo> parseTypeInfos() {
+    typeInfos = new ArrayList<TypeInfo>();
+    iToken = 0;
+    while (iToken < typeInfoTokens.size()) {
+      typeInfos.add(parseType());
+      if (iToken < typeInfoTokens.size()) {
+        Token separator = typeInfoTokens.get(iToken);
+        if (",".equals(separator.text) || ";".equals(separator.text)
+            || ":".equals(separator.text)) {
+          iToken++;
+        } else {
+          throw new IllegalArgumentException(
+              "Error: ',', ':', or ';' expected at position "
+              + separator.position + " from '" + typeInfoString + "' "
+              + typeInfoTokens);
+        }
+      }
+    }
+    return typeInfos;
+  }
+
+  private Token peek() {
+    if (iToken < typeInfoTokens.size()) {
+      return typeInfoTokens.get(iToken);
+    } else {
+      return null;
+    }
+  }
+
+  private Token expect(String item) {
+    return expect(item, null);
+  }
+
+  private Token expect(String item, String alternative) {
+    if (iToken >= typeInfoTokens.size()) {
+      throw new IllegalArgumentException("Error: " + item
+          + " expected at the end of '" + typeInfoString + "'");
+    }
+    Token t = typeInfoTokens.get(iToken);
+    if (item.equals("type")) {
+      if (!ColumnType.LIST_TYPE_NAME.equals(t.text)
+          && !ColumnType.MAP_TYPE_NAME.equals(t.text)
+          && !ColumnType.STRUCT_TYPE_NAME.equals(t.text)
+          && !ColumnType.UNION_TYPE_NAME.equals(t.text)
+          && !ColumnType.PrimitiveTypes.contains(t.text)
+          && !t.text.equals(alternative)) {
+        throw new IllegalArgumentException("Error: " + item
+            + " expected at the position " + t.position + " of '"
+            + typeInfoString + "' but '" + t.text + "' is found.");
+      }
+    } else if (item.equals("name")) {
+      if (!t.isType && !t.text.equals(alternative)) {
+        throw new IllegalArgumentException("Error: " + item
+            + " expected at the position " + t.position + " of '"
+            + typeInfoString + "' but '" + t.text + "' is found.");
+      }
+    } else {
+      if (!item.equals(t.text) && !t.text.equals(alternative)) {
+        throw new IllegalArgumentException("Error: " + item
+            + " expected at the position " + t.position + " of '"
+            + typeInfoString + "' but '" + t.text + "' is found.");
+      }
+    }
+    iToken++;
+    return t;
+  }
+
+  private String[] parseParams() {
+    List<String> params = new LinkedList<String>();
+
+    Token t = peek();
+    if (t != null && t.text.equals("(")) {
+      expect("(");
+
+      // checking for null in the for-loop condition prevents null-ptr 
exception
+      // and allows us to fail more gracefully with a parsing error.
+      for(t = peek(); (t == null) || !t.text.equals(")"); t = expect(",",")")) 
{
+        params.add(expect("name").text);
+      }
+      if (params.size() == 0) {
+        throw new IllegalArgumentException(
+            "type parameters expected for type string " + typeInfoString);
+      }
+    }
+
+    return params.toArray(new String[params.size()]);
+  }
+
+  private TypeInfo parseType() {
+
+    Token t = expect("type");
+
+    // Is this a primitive type?
+    if (ColumnType.PrimitiveTypes.contains(t.text)) {
+      String[] params = parseParams();
+      switch (t.text) {
+      case ColumnType.CHAR_TYPE_NAME:
+      case ColumnType.VARCHAR_TYPE_NAME:
+        if (params == null || params.length == 0) {
+          throw new IllegalArgumentException(t.text
+              + " type is specified without length: " + typeInfoString);
+        }
+
+        int length = 1;
+        if (params.length == 1) {
+          length = Integer.parseInt(params[0]);
+          if (ColumnType.VARCHAR_TYPE_NAME.equals(t.text)) {
+            return 
typeInfoFactory.getPrimitiveTypeInfo(ColumnType.VARCHAR_TYPE_NAME, length);
+          } else {
+            return 
typeInfoFactory.getPrimitiveTypeInfo(ColumnType.CHAR_TYPE_NAME, length);
+          }
+        } else if (params.length > 1) {
+          throw new IllegalArgumentException(
+              "Type " + t.text + " only takes one parameter, but " +
+              params.length + " is seen");
+        }
+
+      case ColumnType.DECIMAL_TYPE_NAME:
+        //TODO do we need to support this? this works only by luck because
+        //standalone-metastore depends on storage-api and HiveDecimal happens 
to be
+        //in storage-api
+        int precision = HiveDecimal.USER_DEFAULT_PRECISION;
+        int scale = HiveDecimal.USER_DEFAULT_SCALE;
+        if (params == null || params.length == 0) {
+          // It's possible that old metadata still refers to "decimal" as a 
column type w/o
+          // precision/scale. In this case, the default (10,0) is assumed. 
Thus, do nothing here.
+        } else if (params.length == 1) {
+          // only precision is specified
+          precision = Integer.valueOf(params[0]);
+        } else if (params.length == 2) {
+          // New metadata always have two parameters.
+          precision = Integer.parseInt(params[0]);
+          scale = Integer.parseInt(params[1]);
+        } else if (params.length > 2) {
+          throw new IllegalArgumentException("Type decimal only takes two 
parameter, but " +
+              params.length + " is seen");
+        }
+        return 
typeInfoFactory.getPrimitiveTypeInfo(ColumnType.DECIMAL_TYPE_NAME, precision, 
scale);
+
+      default:
+        return typeInfoFactory.getPrimitiveTypeInfo(t.text);
+      }
+    }
+
+    // Is this a list type?
+    if (ColumnType.LIST_TYPE_NAME.equals(t.text)) {
+      expect("<");
+      TypeInfo listElementType = parseType();
+      expect(">");
+      return typeInfoFactory.getListTypeInfo(listElementType);
+    }
+
+    // Is this a map type?
+    if (ColumnType.MAP_TYPE_NAME.equals(t.text)) {
+      expect("<");
+      TypeInfo mapKeyType = parseType();
+      expect(",");
+      TypeInfo mapValueType = parseType();
+      expect(">");
+      return typeInfoFactory.getMapTypeInfo(mapKeyType, mapValueType);
+    }
+
+    // Is this a struct type?
+    if (ColumnType.STRUCT_TYPE_NAME.equals(t.text)) {
+      ArrayList<String> fieldNames = new ArrayList<>();
+      ArrayList<TypeInfo> fieldTypeInfos = new ArrayList<>();
+      boolean first = true;
+      do {
+        if (first) {
+          expect("<");
+          first = false;
+        } else {
+          Token separator = expect(">", ",");
+          if (separator.text.equals(">")) {
+            // end of struct
+            break;
+          }
+        }
+        Token name = expect("name",">");
+        if (name.text.equals(">")) {
+          break;
+        }
+        fieldNames.add(name.text);
+        expect(":");
+        fieldTypeInfos.add(parseType());
+      } while (true);
+
+      return typeInfoFactory.getStructTypeInfo(fieldNames, fieldTypeInfos);
+    }
+    // Is this a union type?
+    if (ColumnType.UNION_TYPE_NAME.equals(t.text)) {
+      List<TypeInfo> objectTypeInfos = new ArrayList<>();
+      boolean first = true;
+      do {
+        if (first) {
+          expect("<");
+          first = false;
+        } else {
+          Token separator = expect(">", ",");
+          if (separator.text.equals(">")) {
+            // end of union
+            break;
+          }
+        }
+        objectTypeInfos.add(parseType());
+      } while (true);
+
+      return typeInfoFactory.getUnionTypeInfo(objectTypeInfos);
+    }
+
+    throw new RuntimeException("Internal error parsing position "
+        + t.position + " of '" + typeInfoString + "'");
+  }
+
+  public PrimitiveParts parsePrimitiveParts() {
+    PrimitiveParts parts = new PrimitiveParts();
+    Token t = expect("type");
+    parts.typeName = t.text;
+    parts.typeParams = parseParams();
+    return parts;
+  }
+
+  public static class PrimitiveParts {
+    public String  typeName;
+    public String[] typeParams;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/UnionTypeInfo.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/UnionTypeInfo.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/UnionTypeInfo.java
new file mode 100644
index 0000000..2de835c
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/UnionTypeInfo.java
@@ -0,0 +1,107 @@
+/*
+ * 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.typeinfo;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.ColumnType;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * UnionTypeInfo represents the TypeInfo of an union. A union holds only one
+ * field of the specified fields at any point of time. The fields, a Union can
+ * hold, can have the same or different TypeInfo.
+ *
+ * Always use the TypeInfoFactory to create new TypeInfo objects, instead of
+ * directly creating an instance of this class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class UnionTypeInfo extends TypeInfo implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  private List<TypeInfo> allUnionObjectTypeInfos;
+
+  /**
+   * For java serialization use only.
+   */
+  public UnionTypeInfo() {
+  }
+
+  @Override
+  public String getTypeName() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(ColumnType.UNION_TYPE_NAME + "<");
+    for (int i = 0; i < allUnionObjectTypeInfos.size(); i++) {
+      if (i > 0) {
+        sb.append(",");
+      }
+      sb.append(allUnionObjectTypeInfos.get(i).getTypeName());
+    }
+    sb.append(">");
+    return sb.toString();
+  }
+
+  /**
+   * For java serialization use only.
+   */
+  public void setAllUnionObjectTypeInfos(
+      List<TypeInfo> allUnionObjectTypeInfos) {
+    this.allUnionObjectTypeInfos = allUnionObjectTypeInfos;
+  }
+
+  /**
+   * For TypeInfoFactory use only.
+   */
+  UnionTypeInfo(List<TypeInfo> typeInfos) {
+    allUnionObjectTypeInfos = new ArrayList<TypeInfo>();
+    allUnionObjectTypeInfos.addAll(typeInfos);
+  }
+
+  @Override
+  public MetastoreTypeCategory getCategory() {
+    return MetastoreTypeCategory.UNION;
+  }
+
+  public List<TypeInfo> getAllUnionObjectTypeInfos() {
+    return allUnionObjectTypeInfos;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (!(other instanceof UnionTypeInfo)) {
+      return false;
+    }
+    UnionTypeInfo o = (UnionTypeInfo) other;
+
+    // Compare the field types
+    return o.getAllUnionObjectTypeInfos().equals(getAllUnionObjectTypeInfos());
+  }
+
+  @Override
+  public int hashCode() {
+    return allUnionObjectTypeInfos.hashCode();
+  }
+}

Reply via email to