You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vi...@apache.org on 2018/03/12 03:52:22 UTC

[1/5] hive git commit: HIVE-17580 : Remove dependency of get_fields_with_environment_context API to serde (Vihang Karajgaonkar, reviewed by Alan Gates)

Repository: hive
Updated Branches:
  refs/heads/standalone-metastore c4d22858c -> 40ee74ebc


http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/schema/reader/TestDefaultStorageSchemaReader.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/schema/reader/TestDefaultStorageSchemaReader.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/schema/reader/TestDefaultStorageSchemaReader.java
new file mode 100644
index 0000000..6ba90d6
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/schema/reader/TestDefaultStorageSchemaReader.java
@@ -0,0 +1,598 @@
+/*
+ * 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.metastore.schema.reader;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.DefaultStorageSchemaReader;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.Type;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.utils.AvroSchemaUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+@Category(MetastoreUnitTest.class)
+public class TestDefaultStorageSchemaReader {
+  private static final Logger LOG = LoggerFactory.getLogger(TestDefaultStorageSchemaReader.class);
+  private static final String TEST_DB_NAME = "TEST_DB";
+  private static final String TEST_TABLE_NAME = "TEST_TABLE";
+  private HiveMetaStoreClient client;
+  private Configuration conf;
+  private Warehouse warehouse;
+  private static final int DEFAULT_LIMIT_PARTITION_REQUEST = 100;
+  private static final String AVRO_SERIALIZATION_LIB =
+      "org.apache.hadoop.hive.serde2.avro.AvroSerDe";
+
+  // These schemata are used in other tests
+  static public final String MAP_WITH_PRIMITIVE_VALUE_TYPE = "{\n" +
+      "  \"namespace\": \"testing\",\n" +
+      "  \"name\": \"oneMap\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"aMap\",\n" +
+      "      \"type\":{\"type\":\"map\",\n" +
+      "      \"values\":\"long\"}\n" +
+      "\t}\n" +
+      "  ]\n" +
+      "}";
+  static public final String ARRAY_WITH_PRIMITIVE_ELEMENT_TYPE = "{\n" +
+      "  \"namespace\": \"testing\",\n" +
+      "  \"name\": \"oneArray\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"anArray\",\n" +
+      "      \"type\":{\"type\":\"array\",\n" +
+      "      \"items\":\"string\"}\n" +
+      "\t}\n" +
+      "  ]\n" +
+      "}";
+  public static final String RECORD_SCHEMA = "{\n" +
+      "  \"namespace\": \"testing.test.mctesty\",\n" +
+      "  \"name\": \"oneRecord\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"aRecord\",\n" +
+      "      \"type\":{\"type\":\"record\",\n" +
+      "              \"name\":\"recordWithinARecord\",\n" +
+      "              \"fields\": [\n" +
+      "                 {\n" +
+      "                  \"name\":\"int1\",\n" +
+      "                  \"type\":\"int\"\n" +
+      "                },\n" +
+      "                {\n" +
+      "                  \"name\":\"boolean1\",\n" +
+      "                  \"type\":\"boolean\"\n" +
+      "                },\n" +
+      "                {\n" +
+      "                  \"name\":\"long1\",\n" +
+      "                  \"type\":\"long\"\n" +
+      "                }\n" +
+      "      ]}\n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+  public static final String NULLABLE_RECORD_SCHEMA = "[\"null\", " + RECORD_SCHEMA + "]";
+  public static final String UNION_SCHEMA = "{\n" +
+      "  \"namespace\": \"test.a.rossa\",\n" +
+      "  \"name\": \"oneUnion\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"aUnion\",\n" +
+      "      \"type\":[\"int\", \"string\"]\n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+  public static final String UNION_SCHEMA_2 = "{\n" +
+      "  \"namespace\": \"test.a.rossa\",\n" +
+      "  \"name\": \"oneUnion\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"aUnion\",\n" +
+      "      \"type\":[\"null\", \"int\", \"string\"]\n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+  public static final String UNION_SCHEMA_3 = "{\n" +
+      "  \"namespace\": \"test.a.rossa\",\n" +
+      "  \"name\": \"oneUnion\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"aUnion\",\n" +
+      "      \"type\":[\"float\",\"int\"]\n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+  public static final String UNION_SCHEMA_4 = "{\n" +
+      "  \"namespace\": \"test.a.rossa\",\n" +
+      "  \"name\": \"oneUnion\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"aUnion\",\n" +
+      "      \"type\":[\"int\",\"float\",\"long\"]\n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+  public static final String ENUM_SCHEMA = "{\n" +
+      "  \"namespace\": \"clever.namespace.name.in.space\",\n" +
+      "  \"name\": \"oneEnum\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "   {\n" +
+      "      \"name\":\"baddies\",\n" +
+      "      \"type\":{\"type\":\"enum\",\"name\":\"villians\", \"symbols\": " +
+      "[\"DALEKS\", \"CYBERMEN\", \"SLITHEEN\", \"JAGRAFESS\"]}\n" +
+      "      \n" +
+      "      \n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+  public static final String FIXED_SCHEMA = "{\n" +
+      "  \"namespace\": \"ecapseman\",\n" +
+      "  \"name\": \"oneFixed\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "   {\n" +
+      "      \"name\":\"hash\",\n" +
+      "      \"type\":{\"type\": \"fixed\", \"name\": \"MD5\", \"size\": 16}\n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+  public static final String NULLABLE_STRING_SCHEMA = "{\n" +
+      "  \"type\": \"record\", \n" +
+      "  \"name\": \"nullableUnionTest\",\n" +
+      "  \"fields\" : [\n" +
+      "    {\"name\":\"nullableString\", \"type\":[\"null\", \"string\"]}\n" +
+      "  ]\n" +
+      "}";
+  public static final String MAP_WITH_NULLABLE_PRIMITIVE_VALUE_TYPE_SCHEMA = "{\n" +
+      "  \"namespace\": \"testing\",\n" +
+      "  \"name\": \"mapWithNullableUnionTest\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"aMap\",\n" +
+      "      \"type\":{\"type\":\"map\",\n" +
+      "      \"values\":[\"null\",\"long\"]}\n" +
+      "\t}\n" +
+      "  ]\n" +
+      "}";
+  public static final String NULLABLE_ENUM_SCHEMA = "{\n" +
+      "  \"namespace\": \"clever.namespace.name.in.space\",\n" +
+      "  \"name\": \"nullableUnionTest\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "   {\n" +
+      "      \"name\":\"nullableEnum\",\n" +
+      "      \"type\": [\"null\", {\"type\":\"enum\",\"name\":\"villians\", \"symbols\": " +
+      "[\"DALEKS\", \"CYBERMEN\", \"SLITHEEN\", \"JAGRAFESS\"]}]\n" +
+      "      \n" +
+      "      \n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+  public static final String BYTES_SCHEMA = "{\n" +
+      "  \"type\": \"record\", \n" +
+      "  \"name\": \"bytesTest\",\n" +
+      "  \"fields\" : [\n" +
+      "    {\"name\":\"bytesField\", \"type\":\"bytes\"}\n" +
+      "  ]\n" +
+      "}";
+
+  public static final String KITCHEN_SINK_SCHEMA = "{\n" +
+      "  \"namespace\": \"org.apache.hadoop.hive\",\n" +
+      "  \"name\": \"kitchsink\",\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"fields\": [\n" +
+      "    {\n" +
+      "      \"name\":\"string1\",\n" +
+      "      \"type\":\"string\"\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"string2\",\n" +
+      "      \"type\":\"string\"\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"int1\",\n" +
+      "      \"type\":\"int\"\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"boolean1\",\n" +
+      "      \"type\":\"boolean\"\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"long1\",\n" +
+      "      \"type\":\"long\"\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"float1\",\n" +
+      "      \"type\":\"float\"\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"double1\",\n" +
+      "      \"type\":\"double\"\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"inner_record1\",\n" +
+      "      \"type\":{ \"type\":\"record\",\n" +
+      "               \"name\":\"inner_record1_impl\",\n" +
+      "               \"fields\": [\n" +
+      "                          {\"name\":\"int_in_inner_record1\",\n" +
+      "                           \"type\":\"int\"},\n" +
+      "                          {\"name\":\"string_in_inner_record1\",\n" +
+      "                           \"type\":\"string\"}\n" +
+      "                         ]\n" +
+      "       }\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"enum1\",\n" +
+      "      \"type\":{\"type\":\"enum\", \"name\":\"enum1_values\", " +
+      "\"symbols\":[\"ENUM1_VALUES_VALUE1\",\"ENUM1_VALUES_VALUE2\", \"ENUM1_VALUES_VALUE3\"]}\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"array1\",\n" +
+      "      \"type\":{\"type\":\"array\", \"items\":\"string\"}\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"map1\",\n" +
+      "      \"type\":{\"type\":\"map\", \"values\":\"string\"}\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"union1\",\n" +
+      "      \"type\":[\"float\", \"boolean\", \"string\"]\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"fixed1\",\n" +
+      "      \"type\":{\"type\":\"fixed\", \"name\":\"fourbytes\", \"size\":4}\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"null1\",\n" +
+      "      \"type\":\"null\"\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"UnionNullInt\",\n" +
+      "      \"type\":[\"int\", \"null\"]\n" +
+      "    },\n" +
+      "    {\n" +
+      "      \"name\":\"bytes1\",\n" +
+      "      \"type\":\"bytes\"\n" +
+      "    }\n" +
+      "  ]\n" +
+      "}";
+
+  @Before
+  public void setUp() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+    warehouse = new Warehouse(conf);
+
+    // set some values to use for getting conf. vars
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED, true);
+    conf.set("hive.key1", "value1");
+    conf.set("hive.key2", "http://www.example.com");
+    conf.set("hive.key3", "");
+    conf.set("hive.key4", "0");
+    conf.set("datanucleus.autoCreateTables", "false");
+
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX, 2);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.LIMIT_PARTITION_REQUEST,
+        DEFAULT_LIMIT_PARTITION_REQUEST);
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.STORAGE_SCHEMA_READER_IMPL,
+        DefaultStorageSchemaReader.class.getName());
+    client = createClient();
+  }
+
+  @After
+  public void closeClient() {
+    client.close();
+  }
+
+  private void silentDropDatabase(String dbName) throws TException {
+    try {
+      for (String tableName : client.getTables(dbName, "*")) {
+        client.dropTable(dbName, tableName);
+      }
+      client.dropDatabase(dbName);
+    } catch (NoSuchObjectException | InvalidOperationException e) {
+      // NOP
+    }
+  }
+
+  private HiveMetaStoreClient createClient() throws Exception {
+    try {
+      return new HiveMetaStoreClient(conf);
+    } catch (Throwable e) {
+      System.err.println("Unable to open the metastore");
+      System.err.println(StringUtils.stringifyException(e));
+      throw new Exception(e);
+    }
+  }
+
+  @Test
+  public void testSimpleAvroTable() throws TException, IOException {
+    List<FieldSchema> fields = new ArrayList<>(2);
+    FieldSchema field = new FieldSchema();
+    field.setName("name");
+    field.setType("string");
+    field.setComment("Test name comment");
+    fields.add(field);
+
+    field = new FieldSchema();
+    field.setName("age");
+    field.setType("int");
+    field.setComment("Test age comment");
+    fields.add(field);
+
+    createTable(TEST_DB_NAME, TEST_TABLE_NAME, AVRO_SERIALIZATION_LIB, fields, null);
+    List<FieldSchema> retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    verifyTableFields(fields, retFields, null);
+  }
+
+  private Table createTable(String dbName, String tblName, String serializationLib,
+      List<FieldSchema> fields, Map<String, String> tblProperties) throws TException, IOException {
+    client.dropTable(dbName, tblName);
+    silentDropDatabase(dbName);
+    Database db = new Database();
+    db.setName(dbName);
+    client.createDatabase(db);
+    db = client.getDatabase(dbName);
+    Path dbPath = new Path(db.getLocationUri());
+    FileSystem fs = FileSystem.get(dbPath.toUri(), conf);
+    String typeName = "dummy";
+    client.dropType(typeName);
+    Type typ1 = new Type();
+    typ1.setName(typeName);
+    typ1.setFields(fields);
+    client.createType(typ1);
+
+    Table tbl = new TableBuilder().setDbName(dbName).setTableName(tblName).setCols(typ1.getFields())
+        .setSerdeLib(serializationLib).setTableParams(tblProperties).build();
+    client.createTable(tbl);
+    return client.getTable(dbName, tblName);
+  }
+
+  @Test
+  public void testExternalSchemaAvroTable() throws TException, IOException {
+    //map
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, MAP_WITH_PRIMITIVE_VALUE_TYPE);
+    List<FieldSchema> retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "aMap", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "map<string,bigint>",
+        retFields.get(0).getType());
+    Assert.assertEquals("Unexpected comment of the field", "", retFields.get(0).getComment());
+
+    //list
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME,
+        ARRAY_WITH_PRIMITIVE_ELEMENT_TYPE);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "anArray", retFields.get(0).getName());
+    Assert
+        .assertEquals("Unexpected type of the field", "array<string>", retFields.get(0).getType());
+
+    //struct
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, RECORD_SCHEMA);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "aRecord", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field",
+        "struct<int1:int,boolean1:boolean,long1:bigint>", retFields.get(0).getType());
+
+    //union
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, UNION_SCHEMA);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "aUnion", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "uniontype<int,string>",
+        retFields.get(0).getType());
+
+    //union-2
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, UNION_SCHEMA_2);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "aUnion", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "uniontype<int,string>",
+        retFields.get(0).getType());
+
+    //union_3
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, UNION_SCHEMA_3);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "aUnion", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "uniontype<float,int>",
+        retFields.get(0).getType());
+
+    //union_4
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, UNION_SCHEMA_4);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "aUnion", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "uniontype<int,float,bigint>",
+        retFields.get(0).getType());
+
+    //enum
+    // Enums are one of two Avro types that Hive doesn't have any native support for.
+    // Column names - we lose the enumness of this schema
+    // Column types become string
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, ENUM_SCHEMA);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "baddies", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "string",
+        retFields.get(0).getType());
+
+    // Hive has no concept of Avro's fixed type.  Fixed -> arrays of bytes
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, FIXED_SCHEMA);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "hash", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "binary",
+        retFields.get(0).getType());
+
+    //nullable string
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, NULLABLE_STRING_SCHEMA);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "nullableString", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "string",
+        retFields.get(0).getType());
+
+    //map with nullable value - That Union[T, NULL] is converted to just T, within a Map
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, MAP_WITH_NULLABLE_PRIMITIVE_VALUE_TYPE_SCHEMA);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "aMap", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "map<string,bigint>",
+        retFields.get(0).getType());
+
+    // That Union[T, NULL] is converted to just T.
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, NULLABLE_ENUM_SCHEMA);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "nullableEnum", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "string",
+        retFields.get(0).getType());
+
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, BYTES_SCHEMA);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 1, retFields.size());
+    Assert.assertEquals("Unexpected name of the field", "bytesField", retFields.get(0).getName());
+    Assert.assertEquals("Unexpected type of the field", "binary",
+        retFields.get(0).getType());
+
+    createAvroTableWithExternalSchema(TEST_DB_NAME, TEST_TABLE_NAME, KITCHEN_SINK_SCHEMA);
+    retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    Assert.assertEquals("Unexpected number of fields", 16, retFields.size());
+    //There are 16 fields in this schema. Instead of verifying all we verify the interesting ones
+    //(ones which have not been tested above)
+    Assert
+        .assertEquals("Unexpected name of 8th field", "inner_record1", retFields.get(7).getName());
+    Assert.assertEquals("Unexpected type of the field",
+        "struct<int_in_inner_record1:int,string_in_inner_record1:string>",
+        retFields.get(7).getType());
+
+    Assert.assertEquals("Unexpected field name of the 10th field", "array1",
+        retFields.get(9).getName());
+    Assert.assertEquals("Unexpected field type of the 10th field", "array<string>",
+        retFields.get(9).getType());
+
+    Assert.assertEquals("Unexpected field name of the 11th field", "map1",
+        retFields.get(10).getName());
+    Assert.assertEquals("Unexpected field type of the 11th field", "map<string,string>",
+        retFields.get(10).getType());
+
+    Assert.assertEquals("Unexpected field name of the 12th field", "union1",
+        retFields.get(11).getName());
+    Assert
+        .assertEquals("Unexpected field type of the 12th field", "uniontype<float,boolean,string>",
+            retFields.get(11).getType());
+
+    Assert.assertEquals("Unexpected field name of the 14th field", "null1",
+        retFields.get(13).getName());
+    Assert.assertEquals("Unexpected field type of the 14th field", "void",
+        retFields.get(13).getType());
+
+    Assert.assertEquals("Unexpected field name of the 15th field", "UnionNullInt",
+        retFields.get(14).getName());
+    Assert.assertEquals("Unexpected field type of the 15th field", "int",
+        retFields.get(14).getType());
+  }
+
+  private void createAvroTableWithExternalSchema(String dbName, String tblName, String schemaStr)
+      throws TException, IOException {
+    List<FieldSchema> fields = new ArrayList<>(0);
+    Map<String, String> tblParams = new HashMap<>();
+    tblParams.put(AvroSchemaUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName(), schemaStr);
+    createTable(dbName, tblName, AVRO_SERIALIZATION_LIB, fields, tblParams);
+  }
+
+  @Test
+  public void testSimpleTable() throws TException, IOException {
+    List<FieldSchema> fields = new ArrayList<>(2);
+    FieldSchema field = new FieldSchema();
+    field.setName("name");
+    field.setType("string");
+    field.setComment("Test name comment");
+    fields.add(field);
+
+    field = new FieldSchema();
+    field.setName("age");
+    field.setType("int");
+    field.setComment("Test age comment");
+    fields.add(field);
+
+    createTable(TEST_DB_NAME, TEST_TABLE_NAME, null, fields, null);
+    List<FieldSchema> retFields = client.getFields(TEST_DB_NAME, TEST_TABLE_NAME);
+    verifyTableFields(fields, retFields, null);
+  }
+
+  private void verifyTableFields(List<FieldSchema> expected, List<FieldSchema> actual,
+      String nullCommentText) {
+    Assert.assertEquals(expected.size(), actual.size());
+    int size = expected.size();
+    for (int i = 0; i < size; i++) {
+      FieldSchema expectedField = expected.get(i);
+      FieldSchema actualField = actual.get(i);
+      Assert.assertEquals("Name does not match for field " + (i + 1), expectedField.getName(),
+          actualField.getName());
+      Assert.assertEquals("Type does not match for field " + (i + 1), expectedField.getType(),
+          actualField.getType());
+      String expectedComment = null;
+      if (expectedField.getComment() == null && nullCommentText != null) {
+        expectedComment = nullCommentText;
+      } else {
+        expectedComment = expectedField.getComment();
+      }
+      Assert.assertEquals("Comment does not match for field " + (i + 1), expectedComment,
+          actualField.getComment());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/test/java/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
new file mode 100644
index 0000000..e7ca473
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
@@ -0,0 +1,99 @@
+/*
+ * 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 java.util.Set;
+
+import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertSame;
+
+@Category(MetastoreUnitTest.class)
+public class TestInstanceCache {
+  private static class Foo {
+
+    private int value = 42;
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      Foo foo = (Foo) o;
+
+      return value == foo.value;
+
+    }
+
+    @Override
+    public int hashCode() {
+      return value;
+    }
+  }
+
+  private static class Wrapper<T> {
+    public final T wrapped;
+
+    private Wrapper(T wrapped) {
+      this.wrapped = wrapped;
+    }
+  }
+
+  @Test
+  public void instanceCachesOnlyCreateOneInstance() throws Exception {
+    InstanceCache<Foo, Wrapper<Foo>> ic = new InstanceCache<Foo, Wrapper<Foo>>() {
+      @Override
+      protected Wrapper makeInstance(Foo hv,
+                                     Set<Foo> seenSchemas) {
+        return new Wrapper(hv);
+      }
+    };
+    Foo f1 = new Foo();
+
+    Wrapper fc = ic.retrieve(f1, null);
+    assertSame(f1, fc.wrapped); // Our original foo should be in the wrapper
+
+    Foo f2 = new Foo(); // Different instance, same value
+
+    Wrapper fc2 = ic.retrieve(f2, null);
+    assertSame(fc2,fc); // Since equiv f, should get back first container
+    assertSame(fc2.wrapped, f1);
+  }
+
+  @Test
+  public void instanceCacheReturnsCorrectInstances() throws Exception {
+    InstanceCache<String, Wrapper<String>> ic = new InstanceCache<String, Wrapper<String>>() {
+      @Override
+      protected Wrapper<String> makeInstance(
+          String hv, Set<String> seenSchemas) {
+        return new Wrapper<String>(hv);
+      }
+    };
+
+    Wrapper<String> one = ic.retrieve("one", null);
+    Wrapper<String> two = ic.retrieve("two", null);
+
+    Wrapper<String> anotherOne = ic.retrieve("one", null);
+    assertSame(one, anotherOne);
+
+    Wrapper<String> anotherTwo = ic.retrieve("two", null);
+    assertSame(two, anotherTwo);
+  }
+}


[2/5] hive git commit: HIVE-17580 : Remove dependency of get_fields_with_environment_context API to serde (Vihang Karajgaonkar, reviewed by Alan Gates)

Posted by vi...@apache.org.
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();
+  }
+}


[5/5] hive git commit: HIVE-17580 : Remove dependency of get_fields_with_environment_context API to serde (Vihang Karajgaonkar, reviewed by Alan Gates)

Posted by vi...@apache.org.
HIVE-17580 : Remove dependency of get_fields_with_environment_context API to serde (Vihang Karajgaonkar, reviewed by Alan Gates)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/40ee74eb
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/40ee74eb
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/40ee74eb

Branch: refs/heads/standalone-metastore
Commit: 40ee74ebc9a8b06bd8f3b2d7f625b4da23dbff87
Parents: c4d2285
Author: Vihang Karajgaonkar <vi...@cloudera.com>
Authored: Tue Feb 27 21:32:47 2018 -0800
Committer: Vihang Karajgaonkar <vi...@cloudera.com>
Committed: Sun Mar 11 20:49:37 2018 -0700

----------------------------------------------------------------------
 .../accumulo/serde/AccumuloRowSerializer.java   |   2 +-
 .../hive/contrib/serde2/TypedBytesSerDe.java    |   2 +-
 .../hadoop/hive/hbase/ColumnMappings.java       |  26 +-
 .../hadoop/hive/hbase/HBaseStorageHandler.java  |   2 +-
 .../hcatalog/data/schema/HCatSchemaUtils.java   |   4 +-
 .../hive/llap/io/api/impl/LlapRecordReader.java |   2 +-
 .../metastore/SerDeStorageSchemaReader.java     |   4 +
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |  26 +-
 .../hive/ql/exec/persistence/MapJoinKey.java    |   2 +-
 .../hive/ql/exec/vector/VectorAssignRow.java    |   7 +-
 .../ql/exec/vector/VectorDeserializeRow.java    |   2 +-
 .../hive/ql/exec/vector/VectorExtractRow.java   |   2 +-
 .../hive/ql/exec/vector/VectorSerializeRow.java |   2 +-
 .../ql/exec/vector/VectorizationContext.java    |  16 +-
 .../VectorMapJoinGenerateResultOperator.java    |   2 +-
 .../ql/exec/vector/udf/VectorUDFArgDesc.java    |   2 +-
 .../io/parquet/convert/HiveSchemaConverter.java |  10 +-
 .../io/parquet/convert/HiveStructConverter.java |   7 +-
 .../parquet/read/DataWritableReadSupport.java   |   2 +-
 .../serde/ArrayWritableObjectInspector.java     |   6 +-
 .../ql/io/parquet/serde/ParquetHiveSerDe.java   |   2 +-
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java  |   2 +-
 .../hive/ql/optimizer/ColumnPrunerProcCtx.java  |   4 +-
 .../optimizer/ConstantPropagateProcFactory.java |   4 +-
 ...tedDynPartitionTimeGranularityOptimizer.java |   2 +-
 .../calcite/translator/ExprNodeConverter.java   |   2 +-
 .../calcite/translator/RexNodeConverter.java    |   2 +-
 .../ql/optimizer/pcr/PcrExprProcFactory.java    |   2 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |  14 +-
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |   2 +-
 .../apache/hadoop/hive/ql/parse/ParseUtils.java |   3 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   4 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |   8 +-
 .../hive/ql/plan/ExprNodeConstantDesc.java      |   4 +-
 .../hive/ql/plan/VectorPartitionConversion.java |   4 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |   2 +-
 .../hive/ql/udf/generic/GenericUDAFAverage.java |   2 +-
 .../generic/GenericUDAFBinarySetFunctions.java  |   4 +-
 .../ql/udf/generic/GenericUDAFComputeStats.java |   2 +-
 .../udf/generic/GenericUDAFContextNGrams.java   |  10 +-
 .../ql/udf/generic/GenericUDAFCorrelation.java  |   4 +-
 .../ql/udf/generic/GenericUDAFCovariance.java   |   4 +-
 .../generic/GenericUDAFCovarianceSample.java    |   4 +-
 .../generic/GenericUDAFHistogramNumeric.java    |   4 +-
 .../hive/ql/udf/generic/GenericUDAFStd.java     |   2 +-
 .../ql/udf/generic/GenericUDAFStdSample.java    |   2 +-
 .../hive/ql/udf/generic/GenericUDAFSum.java     |   4 +-
 .../udf/generic/GenericUDAFSumEmptyIsZero.java  |   2 +-
 .../ql/udf/generic/GenericUDAFVariance.java     |   2 +-
 .../udf/generic/GenericUDAFVarianceSample.java  |   2 +-
 .../hive/ql/udf/generic/GenericUDAFnGrams.java  |   8 +-
 .../hive/ql/udf/generic/GenericUDFUtils.java    |   2 +-
 .../hive/ql/exec/vector/TestVectorSerDeRow.java |   2 +-
 .../ql/exec/vector/VectorRandomRowSource.java   |   4 +-
 .../mapjoin/fast/CheckFastRowHashMap.java       |   2 +-
 .../results/clientnegative/avro_decimal.q.out   |   2 +-
 serde/pom.xml                                   |   5 +
 .../apache/hadoop/hive/serde2/SerDeUtils.java   |   1 -
 .../avro/AvroObjectInspectorGenerator.java      |   9 +-
 .../hadoop/hive/serde2/avro/AvroSerializer.java |   6 +-
 .../hadoop/hive/serde2/avro/InstanceCache.java  |  72 ---
 .../serde2/avro/SchemaResolutionProblem.java    |  59 --
 .../hive/serde2/avro/SchemaToHiveTypeInfo.java  |  18 +
 .../hive/serde2/avro/SchemaToTypeInfo.java      | 283 ---------
 .../hive/serde2/avro/TypeInfoToSchema.java      | 283 ---------
 .../fast/BinarySortableDeserializeRead.java     |   2 +-
 .../hive/serde2/fast/DeserializeRead.java       |   2 +-
 .../hadoop/hive/serde2/lazy/LazyFactory.java    |   2 +-
 .../lazy/fast/LazySimpleDeserializeRead.java    |   2 +-
 .../fast/LazyBinaryDeserializeRead.java         |   2 +-
 .../serde2/objectinspector/ObjectInspector.java |  87 ++-
 .../hive/serde2/typeinfo/BaseCharTypeInfo.java  |   5 +
 .../hive/serde2/typeinfo/DecimalTypeInfo.java   |   5 +
 .../hive/serde2/typeinfo/ListTypeInfo.java      |  93 ---
 .../hive/serde2/typeinfo/MapTypeInfo.java       | 109 ----
 .../hive/serde2/typeinfo/PrimitiveTypeInfo.java |  48 +-
 .../hive/serde2/typeinfo/StructTypeInfo.java    | 151 -----
 .../typeinfo/TimestampLocalTZTypeInfo.java      |   5 +
 .../hadoop/hive/serde2/typeinfo/TypeInfo.java   |  85 ---
 .../hive/serde2/typeinfo/TypeInfoFactory.java   |  52 +-
 .../hive/serde2/typeinfo/TypeInfoUtils.java     | 344 +----------
 .../hive/serde2/typeinfo/UnionTypeInfo.java     | 108 ----
 .../hive/serde2/typeinfo/VarcharTypeInfo.java   |   1 -
 .../hive/serde2/SerdeRandomRowSource.java       |   4 +-
 .../avro/TestAvroObjectInspectorGenerator.java  |   7 +-
 .../hive/serde2/avro/TestInstanceCache.java     |  95 ---
 .../hive/serde2/avro/TestSchemaToTypeInfo.java  |   7 +-
 .../binarysortable/TestBinarySortableFast.java  |   2 +-
 .../hive/serde2/lazy/TestLazySimpleFast.java    |   4 +-
 .../serde2/lazybinary/TestLazyBinaryFast.java   |   2 +-
 .../TestStandardObjectInspectors.java           |   2 +-
 .../hive/metastore/AvroStorageSchemaReader.java |  48 ++
 .../hadoop/hive/metastore/ColumnType.java       |   5 +-
 .../metastore/DefaultStorageSchemaReader.java   | 108 +++-
 .../hive/metastore/StorageSchemaReader.java     |   2 +-
 .../utils/AvroFieldSchemaGenerator.java         |  97 +++
 .../hive/metastore/utils/AvroSchemaUtils.java   | 366 ++++++++++++
 .../metastore/utils/StorageSchemaUtils.java     |  37 ++
 .../hive/serde2/avro/AvroSerDeConstants.java    |  45 ++
 .../hadoop/hive/serde2/avro/InstanceCache.java  |  72 +++
 .../serde2/avro/SchemaResolutionProblem.java    |  62 ++
 .../serde2/avro/SchemaToMetastoreTypeInfo.java  |  31 +
 .../hive/serde2/avro/SchemaToTypeInfo.java      | 294 +++++++++
 .../hive/serde2/avro/TypeInfoToSchema.java      | 277 +++++++++
 .../hive/serde2/typeinfo/ITypeInfoFactory.java  |  71 +++
 .../hive/serde2/typeinfo/ListTypeInfo.java      |  94 +++
 .../hive/serde2/typeinfo/MapTypeInfo.java       | 110 ++++
 .../typeinfo/MetastorePrimitiveTypeInfo.java    |  91 +++
 .../serde2/typeinfo/MetastoreTypeCategory.java  |  37 ++
 .../typeinfo/MetastoreTypeInfoFactory.java      | 128 ++++
 .../serde2/typeinfo/MetastoreTypeInfoUtils.java |  59 ++
 .../hive/serde2/typeinfo/StructTypeInfo.java    | 150 +++++
 .../hadoop/hive/serde2/typeinfo/TypeInfo.java   |  82 +++
 .../hive/serde2/typeinfo/TypeInfoParser.java    | 343 +++++++++++
 .../hive/serde2/typeinfo/UnionTypeInfo.java     | 107 ++++
 .../reader/TestDefaultStorageSchemaReader.java  | 598 +++++++++++++++++++
 .../hive/serde2/avro/TestInstanceCache.java     |  99 +++
 117 files changed, 3747 insertions(+), 1885 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloRowSerializer.java
----------------------------------------------------------------------
diff --git a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloRowSerializer.java b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloRowSerializer.java
index 7ad6a45..9072c2e 100644
--- a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloRowSerializer.java
+++ b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloRowSerializer.java
@@ -208,7 +208,7 @@ public class AccumuloRowSerializer {
     TypeInfo rowIdTypeInfo = TypeInfoUtils.getTypeInfoFromTypeString(rowIdMappingType);
 
     if (!rowIdFieldOI.getCategory().equals(ObjectInspector.Category.PRIMITIVE)
-        && rowIdTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE) {
+        && rowIdTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       // we always serialize the String type using the escaped algorithm for LazyString
       writeString(output, SerDeUtils.getJSONString(rowId, rowIdFieldOI),
           PrimitiveObjectInspectorFactory.javaStringObjectInspector);

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/TypedBytesSerDe.java
----------------------------------------------------------------------
diff --git a/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/TypedBytesSerDe.java b/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/TypedBytesSerDe.java
index bb6d779..3d2774f 100644
--- a/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/TypedBytesSerDe.java
+++ b/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/TypedBytesSerDe.java
@@ -125,7 +125,7 @@ public class TypedBytesSerDe extends AbstractSerDe {
 
     // All columns have to be primitive.
     for (int c = 0; c < numColumns; c++) {
-      if (columnTypes.get(c).getCategory() != Category.PRIMITIVE) {
+      if (columnTypes.get(c).getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
         throw new SerDeException(getClass().getName()
             + " only accepts primitive columns, but column[" + c + "] named "
             + columnNames.get(c) + " has category "

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
index f1887b5..e4a4f92 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ColumnMappings.java
@@ -121,9 +121,9 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
       colMap.columnType = columnTypes.get(i);
       if (colMap.qualifierName == null && !colMap.hbaseRowKey && !colMap.hbaseTimestamp) {
         TypeInfo typeInfo = columnTypes.get(i);
-        if ((typeInfo.getCategory() != ObjectInspector.Category.MAP) ||
+        if ((typeInfo.getCategory() != ObjectInspector.Category.MAP.toMetastoreTypeCategory()) ||
             (((MapTypeInfo) typeInfo).getMapKeyTypeInfo().getCategory()
-                != ObjectInspector.Category.PRIMITIVE)) {
+                != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory())) {
 
           throw new SerDeException(
               serdeName + ": hbase column family '" + colMap.familyName
@@ -192,24 +192,24 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
       if (storageInfo == null) {
 
         // use the table default storage specification
-        if (colType.getCategory() == ObjectInspector.Category.PRIMITIVE) {
+        if (colType.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
           if (!colType.getTypeName().equals(serdeConstants.STRING_TYPE_NAME)) {
             colMap.binaryStorage.add(tableBinaryStorage);
           } else {
             colMap.binaryStorage.add(false);
           }
-        } else if (colType.getCategory() == ObjectInspector.Category.MAP) {
+        } else if (colType.getCategory() == ObjectInspector.Category.MAP.toMetastoreTypeCategory()) {
           TypeInfo keyTypeInfo = ((MapTypeInfo) colType).getMapKeyTypeInfo();
           TypeInfo valueTypeInfo = ((MapTypeInfo) colType).getMapValueTypeInfo();
 
-          if (keyTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE &&
+          if (keyTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory() &&
               !keyTypeInfo.getTypeName().equals(serdeConstants.STRING_TYPE_NAME)) {
             colMap.binaryStorage.add(tableBinaryStorage);
           } else {
             colMap.binaryStorage.add(false);
           }
 
-          if (valueTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE &&
+          if (valueTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory() &&
               !valueTypeInfo.getTypeName().equals(serdeConstants.STRING_TYPE_NAME)) {
             colMap.binaryStorage.add(tableBinaryStorage);
           } else {
@@ -223,7 +223,7 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
         // we have a storage specification for a primitive column type
         String storageOption = storageInfo[0];
 
-        if ((colType.getCategory() == ObjectInspector.Category.MAP) ||
+        if ((colType.getCategory() == ObjectInspector.Category.MAP.toMetastoreTypeCategory()) ||
             !(storageOption.equals("-") || "string".startsWith(storageOption) ||
                 "binary".startsWith(storageOption))) {
           throw new SerDeException("Error: A column storage specification is one of the following:"
@@ -232,7 +232,7 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
               + colMap.columnName);
         }
 
-        if (colType.getCategory() == ObjectInspector.Category.PRIMITIVE &&
+        if (colType.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory() &&
             !colType.getTypeName().equals(serdeConstants.STRING_TYPE_NAME)) {
 
           if ("-".equals(storageOption)) {
@@ -252,7 +252,7 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
         String keyStorage = storageInfo[0];
         String valStorage = storageInfo[1];
 
-        if ((colType.getCategory() != ObjectInspector.Category.MAP) ||
+        if ((colType.getCategory() != ObjectInspector.Category.MAP.toMetastoreTypeCategory()) ||
             !(keyStorage.equals("-") || "string".startsWith(keyStorage) ||
                 "binary".startsWith(keyStorage)) ||
             !(valStorage.equals("-") || "string".startsWith(valStorage) ||
@@ -270,7 +270,7 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
         TypeInfo keyTypeInfo = ((MapTypeInfo) colType).getMapKeyTypeInfo();
         TypeInfo valueTypeInfo = ((MapTypeInfo) colType).getMapValueTypeInfo();
 
-        if (keyTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE &&
+        if (keyTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory() &&
             !keyTypeInfo.getTypeName().equals(serdeConstants.STRING_TYPE_NAME)) {
 
           if (keyStorage.equals("-")) {
@@ -284,7 +284,7 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
           colMap.binaryStorage.add(false);
         }
 
-        if (valueTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE &&
+        if (valueTypeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory() &&
             !valueTypeInfo.getTypeName().equals(serdeConstants.STRING_TYPE_NAME)) {
           if (valStorage.equals("-")) {
             colMap.binaryStorage.add(tableBinaryStorage);
@@ -405,11 +405,11 @@ public class ColumnMappings implements Iterable<ColumnMappings.ColumnMapping> {
     }
 
     public boolean isCategory(ObjectInspector.Category category) {
-      return columnType.getCategory() == category;
+      return columnType.getCategory() == category.toMetastoreTypeCategory();
     }
 
     public boolean isCategory(PrimitiveCategory category) {
-      return columnType.getCategory() == ObjectInspector.Category.PRIMITIVE &&
+      return columnType.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory() &&
           ((PrimitiveTypeInfo)columnType).getPrimitiveCategory() == category;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
index 8c9271c..94b4cdb 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
@@ -425,7 +425,7 @@ public class HBaseStorageHandler extends DefaultStorageHandler
         continue;
       }
       TypeInfo typeInfo = searchConditions.get(0).getColumnDesc().getTypeInfo();
-      if (typeInfo.getCategory() == Category.PRIMITIVE && PrimitiveObjectInspectorUtils.getPrimitiveGrouping(
+      if (typeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory() && PrimitiveObjectInspectorUtils.getPrimitiveGrouping(
               ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory()) == PrimitiveGrouping.NUMERIC_GROUP) {
         // If the predicate is on a numeric column, and it specifies an
         // open range e.g. key < 20 , we do not support conversion, as negative

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/schema/HCatSchemaUtils.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/schema/HCatSchemaUtils.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/schema/HCatSchemaUtils.java
index 999abcb..63bbada 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/schema/HCatSchemaUtils.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/schema/HCatSchemaUtils.java
@@ -116,7 +116,7 @@ public class HCatSchemaUtils {
   }
 
   private static HCatFieldSchema getHCatFieldSchema(String fieldName, TypeInfo fieldTypeInfo, String comment) throws HCatException {
-    Category typeCategory = fieldTypeInfo.getCategory();
+    Category typeCategory = Category.fromMetastoreTypeCategory(fieldTypeInfo.getCategory());
     HCatFieldSchema hCatFieldSchema;
     if (Category.PRIMITIVE == typeCategory) {
       hCatFieldSchema = new HCatFieldSchema(fieldName, (PrimitiveTypeInfo)fieldTypeInfo, comment);
@@ -157,7 +157,7 @@ public class HCatSchemaUtils {
   }
 
   public static HCatSchema getHCatSchema(TypeInfo typeInfo) throws HCatException {
-    Category typeCategory = typeInfo.getCategory();
+    Category typeCategory = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
     HCatSchema hCatSchema;
     if (Category.PRIMITIVE == typeCategory) {
       hCatSchema = getStructSchemaBuilder().addField(new HCatFieldSchema(null, (PrimitiveTypeInfo)typeInfo, null)).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
index d252279..1c4c418 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
@@ -219,7 +219,7 @@ class LlapRecordReader
     double totalWeight = 0;
     for (TypeInfo ti : typeInfos) {
       int colWeight = 1;
-      if (ti.getCategory() != Category.PRIMITIVE) {
+      if (ti.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
         colWeight = COL_WEIGHT_COMPLEX;
       } else {
         PrimitiveTypeInfo pti = (PrimitiveTypeInfo)ti;

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/metastore/src/java/org/apache/hadoop/hive/metastore/SerDeStorageSchemaReader.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/SerDeStorageSchemaReader.java b/metastore/src/java/org/apache/hadoop/hive/metastore/SerDeStorageSchemaReader.java
index 59bcd5c..05e1078 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/SerDeStorageSchemaReader.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/SerDeStorageSchemaReader.java
@@ -27,6 +27,10 @@ import org.apache.hadoop.hive.metastore.utils.StringUtils;
 
 import java.util.List;
 
+/**
+ * In order to use this Storage schema reader you should add the hive-serde jar in the classpath
+ * of the metastore.
+ */
 public class SerDeStorageSchemaReader implements StorageSchemaReader {
   @Override
   public List<FieldSchema> readSchema(Table tbl, EnvironmentContext envContext, Configuration conf)

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index f7801bb..d24803c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -741,7 +741,8 @@ public final class FunctionRegistry {
     if (a.equals(b)) {
       return a;
     }
-    if (a.getCategory() != Category.PRIMITIVE || b.getCategory() != Category.PRIMITIVE) {
+    if (a.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()
+        || b.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
       return null;
     }
     PrimitiveCategory pcA = ((PrimitiveTypeInfo)a).getPrimitiveCategory();
@@ -794,7 +795,8 @@ public final class FunctionRegistry {
     if (a.equals(b)) {
       return a;
     }
-    if (a.getCategory() != Category.PRIMITIVE || b.getCategory() != Category.PRIMITIVE) {
+    if (a.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()
+        || b.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
       return null;
     }
     PrimitiveCategory pcA = ((PrimitiveTypeInfo)a).getPrimitiveCategory();
@@ -849,7 +851,8 @@ public final class FunctionRegistry {
   }
 
   public static PrimitiveCategory getPrimitiveCommonCategory(TypeInfo a, TypeInfo b) {
-    if (a.getCategory() != Category.PRIMITIVE || b.getCategory() != Category.PRIMITIVE) {
+    if (a.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()
+        || b.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
       return null;
     }
 
@@ -928,7 +931,8 @@ public final class FunctionRegistry {
       return getTypeInfoForPrimitiveCategory((PrimitiveTypeInfo)a, (PrimitiveTypeInfo)b, commonCat);
     }
     // It is not primitive; check if it is a struct and we can infer a common class
-    if (a.getCategory() == Category.STRUCT && b.getCategory() == Category.STRUCT) {
+    if (a.getCategory() == Category.STRUCT.toMetastoreTypeCategory()
+        && b.getCategory() == Category.STRUCT.toMetastoreTypeCategory()) {
       return getCommonClassForStruct((StructTypeInfo)a, (StructTypeInfo)b);
     }
     return null;
@@ -1109,8 +1113,8 @@ public final class FunctionRegistry {
       // passing null matches everything
       return 0;
     }
-    if (argumentPassed.getCategory().equals(Category.LIST)
-        && argumentAccepted.getCategory().equals(Category.LIST)) {
+    if (argumentPassed.getCategory().equals(Category.LIST.toMetastoreTypeCategory())
+        && argumentAccepted.getCategory().equals(Category.LIST.toMetastoreTypeCategory())) {
       // lists are compatible if and only-if the elements are compatible
       TypeInfo argumentPassedElement = ((ListTypeInfo) argumentPassed)
           .getListElementTypeInfo();
@@ -1118,8 +1122,8 @@ public final class FunctionRegistry {
           .getListElementTypeInfo();
       return matchCost(argumentPassedElement, argumentAcceptedElement, exact);
     }
-    if (argumentPassed.getCategory().equals(Category.MAP)
-        && argumentAccepted.getCategory().equals(Category.MAP)) {
+    if (argumentPassed.getCategory().equals(Category.MAP.toMetastoreTypeCategory())
+        && argumentAccepted.getCategory().equals(Category.MAP.toMetastoreTypeCategory())) {
       // lists are compatible if and only-if the elements are compatible
       TypeInfo argumentPassedKey = ((MapTypeInfo) argumentPassed)
           .getMapKeyTypeInfo();
@@ -1173,8 +1177,8 @@ public final class FunctionRegistry {
           // Check the affinity of the argument passed in with the accepted argument,
           // based on the PrimitiveGrouping
           TypeInfo passedType = argsPassedIter.next();
-          if (acceptedType.getCategory() == Category.PRIMITIVE
-              && passedType.getCategory() == Category.PRIMITIVE) {
+          if (acceptedType.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()
+              && passedType.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
             PrimitiveGrouping acceptedPg = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(
                 ((PrimitiveTypeInfo) acceptedType).getPrimitiveCategory());
             PrimitiveGrouping passedPg = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(
@@ -1306,7 +1310,7 @@ public final class FunctionRegistry {
 
           boolean acceptedIsPrimitive = false;
           PrimitiveCategory acceptedPrimCat = PrimitiveCategory.UNKNOWN;
-          if (accepted.getCategory() == Category.PRIMITIVE) {
+          if (accepted.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
             acceptedIsPrimitive = true;
             acceptedPrimCat = ((PrimitiveTypeInfo) accepted).getPrimitiveCategory();
           }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
index 6504a5f..8685b8f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
@@ -94,7 +94,7 @@ public abstract class MapJoinKey {
   }
 
   public static boolean isSupportedField(TypeInfo typeInfo) {
-    if (typeInfo.getCategory() != Category.PRIMITIVE) return false; // not supported
+    if (typeInfo.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) return false; // not supported
     PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) typeInfo;
     PrimitiveCategory pc = primitiveTypeInfo.getPrimitiveCategory();
     if (!SUPPORTED_PRIMITIVES.contains(pc)) return false; // not supported

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
index e96619c..dd0a2aa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.StandardUnionObjectInspecto
 import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MetastoreTypeCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo;
 import org.slf4j.Logger;
@@ -141,7 +142,7 @@ public class VectorAssignRow {
     isConvert[logicalColumnIndex] = false;
     projectionColumnNums[logicalColumnIndex] = projectionColumnNum;
     targetTypeInfos[logicalColumnIndex] = typeInfo;
-    if (typeInfo.getCategory() == Category.PRIMITIVE) {
+    if (typeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
       final PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) typeInfo;
       final PrimitiveCategory primitiveCategory = primitiveTypeInfo.getPrimitiveCategory();
       switch (primitiveCategory) {
@@ -164,7 +165,7 @@ public class VectorAssignRow {
    */
   private void initConvertSourceEntry(int logicalColumnIndex, TypeInfo convertSourceTypeInfo) {
     isConvert[logicalColumnIndex] = true;
-    final Category convertSourceCategory = convertSourceTypeInfo.getCategory();
+    final Category convertSourceCategory = Category.fromMetastoreTypeCategory(convertSourceTypeInfo.getCategory());
     convertSourceOI[logicalColumnIndex] =
         TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(convertSourceTypeInfo);
 
@@ -654,7 +655,7 @@ public class VectorAssignRow {
       TypeInfo targetTypeInfo, ObjectInspector sourceObjectInspector,
       Writable convertTargetWritable, Object object) {
 
-    final Category targetCategory = targetTypeInfo.getCategory();
+    final Category targetCategory = Category.fromMetastoreTypeCategory(targetTypeInfo.getCategory());
     if (targetCategory == null) {
       /*
        * This is a column that we don't want (i.e. not included) -- we are done.

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
index 8ea625e..06d9913 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
@@ -265,7 +265,7 @@ public final class VectorDeserializeRow<T extends DeserializeRead> {
   }
 
   private Field allocateComplexField(TypeInfo sourceTypeInfo) {
-    final Category category = sourceTypeInfo.getCategory();
+    final Category category = Category.fromMetastoreTypeCategory(sourceTypeInfo.getCategory());
     switch (category) {
     case LIST:
       {

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
index 681d9ca..b296f63 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
@@ -201,7 +201,7 @@ public class VectorExtractRow {
       return null;
     }
 
-    final Category category = typeInfo.getCategory();
+    final Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
     switch (category) {
     case PRIMITIVE:
       {

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
index cb2efb7..22c8ad5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
@@ -106,7 +106,7 @@ public final class VectorSerializeRow<T extends SerializeWrite> {
 
   private Field createField(TypeInfo typeInfo) {
     final Field field = new Field();
-    final Category category = typeInfo.getCategory();
+    final Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
     field.category = category;
     field.typeInfo = typeInfo;
     if (category == Category.PRIMITIVE) {

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 8264e8a..b2e0ba6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -634,7 +634,7 @@ public class VectorizationContext {
       case FILTER:
         // Evaluate the column as a boolean, converting if necessary.
         TypeInfo typeInfo = exprDesc.getTypeInfo();
-        if (typeInfo.getCategory() == Category.PRIMITIVE &&
+        if (typeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory() &&
             ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory() == PrimitiveCategory.BOOLEAN) {
           expr = new SelectColumnIsTrue(columnNum);
         } else {
@@ -882,11 +882,11 @@ public class VectorizationContext {
       }
     } else if (genericUdf instanceof GenericUDFIn) {
       TypeInfo colTi = children.get(0).getTypeInfo();
-      if (colTi.getCategory() != Category.PRIMITIVE) {
+      if (colTi.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
         return colTi; // Handled later, only struct will be supported.
       }
       TypeInfo opTi = GenericUDFUtils.deriveInType(children);
-      if (opTi == null || opTi.getCategory() != Category.PRIMITIVE) {
+      if (opTi == null || opTi.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
         throw new HiveException("Cannot vectorize IN() - common type is " + opTi);
       }
       if (((PrimitiveTypeInfo)colTi).getPrimitiveCategory() !=
@@ -2162,7 +2162,7 @@ public class VectorizationContext {
     for (int f = 0; f < fieldCount; f++) {
       TypeInfo fieldTypeInfo = fieldTypeInfos.get(f);
       // Only primitive fields supports for now.
-      if (fieldTypeInfo.getCategory() != Category.PRIMITIVE) {
+      if (fieldTypeInfo.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
         return null;
       }
 
@@ -2293,7 +2293,7 @@ public class VectorizationContext {
     String colType = colExpr.getTypeString();
     colType = VectorizationContext.mapTypeNameSynonyms(colType);
     TypeInfo colTypeInfo = TypeInfoUtils.getTypeInfoFromTypeString(colType);
-    Category category = colTypeInfo.getCategory();
+    Category category = Category.fromMetastoreTypeCategory(colTypeInfo.getCategory());
     if (category == Category.STRUCT) {
       return getStructInExpression(childExpr, colExpr, colTypeInfo, inChildren, mode, returnType);
     } else if (category != Category.PRIMITIVE) {
@@ -3010,7 +3010,7 @@ public class VectorizationContext {
 
       // Is output type a BOOLEAN?
       TypeInfo resultTypeInfo = expr.getTypeInfo();
-      if (resultTypeInfo.getCategory() == Category.PRIMITIVE &&
+      if (resultTypeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory() &&
           ((PrimitiveTypeInfo) resultTypeInfo).getPrimitiveCategory() == PrimitiveCategory.BOOLEAN) {
         isFilter = true;
       } else {
@@ -3285,13 +3285,13 @@ public class VectorizationContext {
   static String getScratchName(TypeInfo typeInfo) throws HiveException {
     // For now, leave DECIMAL precision/scale in the name so DecimalColumnVector scratch columns
     // don't need their precision/scale adjusted...
-    if (typeInfo.getCategory() == Category.PRIMITIVE &&
+    if (typeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory() &&
         ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory() == PrimitiveCategory.DECIMAL) {
       return typeInfo.getTypeName();
     }
 
     // And, for Complex Types, also leave the children types in place...
-    if (typeInfo.getCategory() != Category.PRIMITIVE) {
+    if (typeInfo.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
       return typeInfo.getTypeName();
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
index 92ec1ee..fbba720 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
@@ -447,7 +447,7 @@ public abstract class VectorMapJoinGenerateResultOperator extends VectorMapJoinC
     for (int i = 0; i < projectionSize; i++) {
       int projectedColumn = projectedColumns.get(i);
       if (batch.cols[projectedColumn] != null &&
-          inputObjInspectorsTypeInfos[i].getCategory() == Category.PRIMITIVE) {
+          inputObjInspectorsTypeInfos[i].getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
         // Only columns present in the batch and non-complex types.
         typeInfoList.add(inputObjInspectorsTypeInfos[i]);
         noNullsProjectionList.add(projectedColumn);

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFArgDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFArgDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFArgDesc.java
index 69a2bef..8ea5f27 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFArgDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFArgDesc.java
@@ -54,7 +54,7 @@ public class VectorUDFArgDesc implements Serializable {
   public void setConstant(ExprNodeConstantDesc expr) {
     isConstant = true;
     if (expr != null) {
-      if (expr.getTypeInfo().getCategory() == Category.PRIMITIVE) {
+      if (expr.getTypeInfo().getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
         PrimitiveCategory primitiveCategory = ((PrimitiveTypeInfo) expr.getTypeInfo())
             .getPrimitiveCategory();
         if (primitiveCategory == PrimitiveCategory.VOID) {

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java
index 302321c..7111534 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java
@@ -58,7 +58,7 @@ public class HiveSchemaConverter {
 
   private static Type convertType(final String name, final TypeInfo typeInfo,
                                   final Repetition repetition) {
-    if (typeInfo.getCategory().equals(Category.PRIMITIVE)) {
+    if (typeInfo.getCategory().equals(Category.PRIMITIVE.toMetastoreTypeCategory())) {
       if (typeInfo.equals(TypeInfoFactory.stringTypeInfo)) {
         return Types.primitive(PrimitiveTypeName.BINARY, repetition).as(OriginalType.UTF8)
           .named(name);
@@ -107,13 +107,13 @@ public class HiveSchemaConverter {
       } else {
         throw new IllegalArgumentException("Unknown type: " + typeInfo);
       }
-    } else if (typeInfo.getCategory().equals(Category.LIST)) {
+    } else if (typeInfo.getCategory().equals(Category.LIST.toMetastoreTypeCategory())) {
       return convertArrayType(name, (ListTypeInfo) typeInfo);
-    } else if (typeInfo.getCategory().equals(Category.STRUCT)) {
+    } else if (typeInfo.getCategory().equals(Category.STRUCT.toMetastoreTypeCategory())) {
       return convertStructType(name, (StructTypeInfo) typeInfo);
-    } else if (typeInfo.getCategory().equals(Category.MAP)) {
+    } else if (typeInfo.getCategory().equals(Category.MAP.toMetastoreTypeCategory())) {
       return convertMapType(name, (MapTypeInfo) typeInfo);
-    } else if (typeInfo.getCategory().equals(Category.UNION)) {
+    } else if (typeInfo.getCategory().equals(Category.UNION.toMetastoreTypeCategory())) {
       throw new UnsupportedOperationException("Union type not implemented");
     } else {
       throw new IllegalArgumentException("Unknown type: " + typeInfo);

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
index bdffdf4..beb75be 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveStructConverter.java
@@ -77,7 +77,8 @@ public class HiveStructConverter extends HiveGroupConverter {
     converters = new Converter[selectedFieldCount];
     this.repeatedConverters = new ArrayList<Repeated>();
 
-    if (hiveTypeInfo != null && hiveTypeInfo.getCategory().equals(ObjectInspector.Category.STRUCT)) {
+    if (hiveTypeInfo != null && hiveTypeInfo.getCategory()
+        .equals(ObjectInspector.Category.STRUCT.toMetastoreTypeCategory())) {
       this.hiveFieldNames = ((StructTypeInfo) hiveTypeInfo).getAllStructFieldNames();
       this.hiveFieldTypeInfos = ((StructTypeInfo) hiveTypeInfo).getAllStructFieldTypeInfos();
     }
@@ -126,9 +127,9 @@ public class HiveStructConverter extends HiveGroupConverter {
   private TypeInfo getFieldTypeIgnoreCase(TypeInfo hiveTypeInfo, String fieldName, int fieldIndex) {
     if (hiveTypeInfo == null) {
       return null;
-    } else if (hiveTypeInfo.getCategory().equals(ObjectInspector.Category.STRUCT)) {
+    } else if (hiveTypeInfo.getCategory().equals(ObjectInspector.Category.STRUCT.toMetastoreTypeCategory())) {
       return getStructFieldTypeInfo(fieldName, fieldIndex);
-    } else if (hiveTypeInfo.getCategory().equals(ObjectInspector.Category.MAP)) {
+    } else if (hiveTypeInfo.getCategory().equals(ObjectInspector.Category.MAP.toMetastoreTypeCategory())) {
       //This cover the case where hive table may have map<key, value> but the data file is
       // of type array<struct<value1, value2>>
       //Using index in place of type name.

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
index 7f2a684..f69559c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
@@ -146,7 +146,7 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
           .named(fieldType.getName());
       case LIST:
         TypeInfo elemType = ((ListTypeInfo) colType).getListElementTypeInfo();
-        if (elemType.getCategory() == ObjectInspector.Category.STRUCT) {
+        if (elemType.getCategory() == ObjectInspector.Category.STRUCT.toMetastoreTypeCategory()) {
           Type subFieldType = fieldType.asGroupType().getType(0);
           if (!subFieldType.isPrimitive()) {
             String subFieldName = subFieldType.getName();

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java
index d83376d..9f915a2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java
@@ -107,12 +107,12 @@ public class ArrayWritableObjectInspector extends SettableStructObjectInspector
       return ParquetPrimitiveInspectorFactory.parquetStringInspector;
     }  else if (typeInfo instanceof DecimalTypeInfo) {
       return PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector((DecimalTypeInfo) typeInfo);
-    } else if (typeInfo.getCategory().equals(Category.STRUCT)) {
+    } else if (typeInfo.getCategory().equals(Category.STRUCT.toMetastoreTypeCategory())) {
       return new ArrayWritableObjectInspector(false, (StructTypeInfo) typeInfo, (StructTypeInfo) prunedTypeInfo);
-    } else if (typeInfo.getCategory().equals(Category.LIST)) {
+    } else if (typeInfo.getCategory().equals(Category.LIST.toMetastoreTypeCategory())) {
       final TypeInfo subTypeInfo = ((ListTypeInfo) typeInfo).getListElementTypeInfo();
       return new ParquetHiveArrayInspector(getObjectInspector(subTypeInfo, null));
-    } else if (typeInfo.getCategory().equals(Category.MAP)) {
+    } else if (typeInfo.getCategory().equals(Category.MAP.toMetastoreTypeCategory())) {
       final TypeInfo keyTypeInfo = ((MapTypeInfo) typeInfo).getMapKeyTypeInfo();
       final TypeInfo valueTypeInfo = ((MapTypeInfo) typeInfo).getMapValueTypeInfo();
       if (keyTypeInfo.equals(TypeInfoFactory.stringTypeInfo) || keyTypeInfo.equals(TypeInfoFactory.byteTypeInfo)

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java
index e0018a5..86de2b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java
@@ -254,7 +254,7 @@ public class ParquetHiveSerDe extends AbstractSerDe {
       this.selected = new boolean[typeInfo.getAllStructFieldTypeInfos().size()];
       for (int i = 0; i < typeInfo.getAllStructFieldTypeInfos().size(); ++i) {
         TypeInfo ti = typeInfo.getAllStructFieldTypeInfos().get(i);
-        if (ti.getCategory() == Category.STRUCT) {
+        if (ti.getCategory() == Category.STRUCT.toMetastoreTypeCategory()) {
           this.children.put(typeInfo.getAllStructFieldNames().get(i).toLowerCase(),
               new PrunedStructTypeInfo((StructTypeInfo) ti));
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
index 27fe828..0164085 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
@@ -112,7 +112,7 @@ public class ConvertAstToSearchArg {
    */
   private static BoxType getType(ExprNodeDesc expr) {
     TypeInfo type = expr.getTypeInfo();
-    if (type.getCategory() == ObjectInspector.Category.PRIMITIVE) {
+    if (type.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       switch (((PrimitiveTypeInfo) type).getPrimitiveCategory()) {
         case BYTE:
         case SHORT:

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
index c2a2fb1..7eae2a7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
@@ -265,8 +265,8 @@ public class ColumnPrunerProcCtx implements NodeProcessorCtx {
     // Check cases for arr[i].f and map[key].v
     // For these we should not generate paths like arr.f or map.v
     // Otherwise we would have a mismatch between type info and path
-    if (ti.getCategory() != ObjectInspector.Category.LIST
-        && ti.getCategory() != ObjectInspector.Category.MAP) {
+    if (ti.getCategory() != ObjectInspector.Category.LIST.toMetastoreTypeCategory()
+        && ti.getCategory() != ObjectInspector.Category.MAP.toMetastoreTypeCategory()) {
       fn.addFieldNodes(pathToRoot);
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
index 54d52f8..40a3c39 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
@@ -912,7 +912,7 @@ public final class ConstantPropagateProcFactory {
             return null;
           }
         }
-        if (constant.getTypeInfo().getCategory() != Category.PRIMITIVE) {
+        if (constant.getTypeInfo().getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
           // nested complex types cannot be folded cleanly
           return null;
         }
@@ -932,7 +932,7 @@ public final class ConstantPropagateProcFactory {
           return null;
         }
         ExprNodeConstantDesc constant = (ExprNodeConstantDesc) evaluatedFn;
-        if (constant.getTypeInfo().getCategory() != Category.PRIMITIVE) {
+        if (constant.getTypeInfo().getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
           // nested complex types cannot be folded cleanly
           return null;
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
index e3dee93..ab5edde 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
@@ -229,7 +229,7 @@ public class SortedDynPartitionTimeGranularityOptimizer extends Transform {
         ExprNodeColumnDesc columnDesc = new ExprNodeColumnDesc(ci);
         descs.add(columnDesc);
         colNames.add(columnDesc.getExprString());
-        if (columnDesc.getTypeInfo().getCategory() == ObjectInspector.Category.PRIMITIVE
+        if (columnDesc.getTypeInfo().getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()
                 && ((PrimitiveTypeInfo) columnDesc.getTypeInfo()).getPrimitiveCategory() == PrimitiveCategory.TIMESTAMPLOCALTZ) {
           if (timestampPos != -1) {
             throw new SemanticException("Multiple columns with timestamp with local time-zone type on query result; "

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
index 12af94e..872fc41 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
@@ -151,7 +151,7 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
     String child = fieldAccess.getField().getName();
     TypeInfo parentType = parent.getTypeInfo();
     // Allow accessing a field of list element structs directly from a list
-    boolean isList = (parentType.getCategory() == ObjectInspector.Category.LIST);
+    boolean isList = (parentType.getCategory() == ObjectInspector.Category.LIST.toMetastoreTypeCategory());
     if (isList) {
       parentType = ((ListTypeInfo) parentType).getListElementTypeInfo();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
index 39ff591..e04b357 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
@@ -248,7 +248,7 @@ public class RexNodeConverter {
     GenericUDF tgtUdf = func.getGenericUDF();
 
     boolean isNumeric = (tgtUdf instanceof GenericUDFBaseBinary
-        && func.getTypeInfo().getCategory() == Category.PRIMITIVE
+        && func.getTypeInfo().getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()
         && (PrimitiveGrouping.NUMERIC_GROUP == PrimitiveObjectInspectorUtils.getPrimitiveGrouping(
             ((PrimitiveTypeInfo) func.getTypeInfo()).getPrimitiveCategory())));
     boolean isCompare = !isNumeric && tgtUdf instanceof GenericUDFBaseCompare;

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
index ea042bf..7c8d40d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
@@ -306,7 +306,7 @@ public final class PcrExprProcFactory {
         return new NodeInfoWrapper(WalkState.UNKNOWN, null, getOutExpr(fd, nodeOutputs));
       }
 
-      if (has_part_col && fd.getTypeInfo().getCategory() == Category.PRIMITIVE) {
+      if (has_part_col && fd.getTypeInfo().getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
         //  we need to evaluate result for every pruned partition
         if (fd.getTypeInfo().equals(TypeInfoFactory.booleanTypeInfo)) {
           // if the return type of the GenericUDF is boolean and all partitions agree on

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 190771e..c72b2ac 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -2557,7 +2557,7 @@ public class Vectorizer implements PhysicalPlanResolver {
 
           // COUNT, DENSE_RANK, and RANK do not care about column types.  The rest do.
           TypeInfo typeInfo = exprNodeDesc.getTypeInfo();
-          Category category = typeInfo.getCategory();
+          Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
           boolean isSupportedType;
           if (category != Category.PRIMITIVE) {
             isSupportedType = false;
@@ -2664,7 +2664,7 @@ public class Vectorizer implements PhysicalPlanResolver {
     }
     if (desc.getChildren() != null) {
       if (isInExpression
-          && desc.getChildren().get(0).getTypeInfo().getCategory() == Category.STRUCT) {
+          && desc.getChildren().get(0).getTypeInfo().getCategory() == Category.STRUCT.toMetastoreTypeCategory()) {
         // Don't restrict child expressions for projection.
         // Always use loose FILTER mode.
         if (!validateStructInExpression(desc, expressionTitle, VectorExpressionDescriptor.Mode.FILTER)) {
@@ -2688,7 +2688,7 @@ public class Vectorizer implements PhysicalPlanResolver {
       String expressionTitle, VectorExpressionDescriptor.Mode mode) {
     for (ExprNodeDesc d : desc.getChildren()) {
       TypeInfo typeInfo = d.getTypeInfo();
-      if (typeInfo.getCategory() != Category.STRUCT) {
+      if (typeInfo.getCategory() != Category.STRUCT.toMetastoreTypeCategory()) {
         return false;
       }
       StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo;
@@ -2699,7 +2699,7 @@ public class Vectorizer implements PhysicalPlanResolver {
       final int fieldCount = fieldTypeInfos.size();
       for (int f = 0; f < fieldCount; f++) {
         TypeInfo fieldTypeInfo = fieldTypeInfos.get(f);
-        Category category = fieldTypeInfo.getCategory();
+        Category category = Category.fromMetastoreTypeCategory(fieldTypeInfo.getCategory());
         if (category != Category.PRIMITIVE) {
           setExpressionIssue(expressionTitle,
               "Cannot vectorize struct field " + fieldNames.get(f)
@@ -2788,7 +2788,7 @@ public class Vectorizer implements PhysicalPlanResolver {
 
     if (!result) {
       TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(type);
-      if (typeInfo.getCategory() != Category.PRIMITIVE) {
+      if (typeInfo.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
         if (allowComplex) {
           return true;
         }
@@ -2808,7 +2808,7 @@ public class Vectorizer implements PhysicalPlanResolver {
 
     if (!result) {
       TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(type);
-      if (typeInfo.getCategory() != Category.PRIMITIVE) {
+      if (typeInfo.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
         if (allowComplex && isVectorizationComplexTypesEnabled) {
           return null;
         } else if (!allowComplex) {
@@ -3131,7 +3131,7 @@ public class Vectorizer implements PhysicalPlanResolver {
       // same check used in HashTableLoader.
       if (!MapJoinKey.isSupportedField(typeInfo)) {
         supportsKeyTypes = false;
-        Category category = typeInfo.getCategory();
+        Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
         notSupportedKeyTypes.add(
             (category != Category.PRIMITIVE ? category.toString() :
               ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory().toString()));

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
index c97e2a9..2dfcc16 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
@@ -228,7 +228,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
         if (colName.equalsIgnoreCase(col.getName())) {
           String type = col.getType();
           TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(type);
-          if (typeInfo.getCategory() != ObjectInspector.Category.PRIMITIVE) {
+          if (typeInfo.getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
             logTypeWarning(colName, type);
             colNames.remove(colName);
           } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
index 89e8412..e1710fd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hive.ql.parse.CalcitePlanner.ASTSearcher;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MetastoreTypeInfoUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
@@ -149,7 +150,7 @@ public final class ParseUtils {
   public static ExprNodeDesc createConversionCast(ExprNodeDesc column, PrimitiveTypeInfo tableFieldTypeInfo)
       throws SemanticException {
     // Get base type, since type string may be parameterized
-    String baseType = TypeInfoUtils.getBaseName(tableFieldTypeInfo.getTypeName());
+    String baseType = MetastoreTypeInfoUtils.getBaseName(tableFieldTypeInfo.getTypeName());
 
     // If the type cast UDF is for a parameterized type, then it should implement
     // the SettableUDF interface so that we can pass in the params.

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 5c96653..ee860ab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -7585,7 +7585,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         if (!tableFieldTypeInfo.equals(rowFieldTypeInfo)) {
           // need to do some conversions here
           converted = true;
-          if (tableFieldTypeInfo.getCategory() != Category.PRIMITIVE) {
+          if (tableFieldTypeInfo.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
             // cannot convert to complex types
             column = null;
           } else {
@@ -7850,7 +7850,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
       if (convert && !tableFieldTypeInfo.equals(rowFieldTypeInfo)) {
         // need to do some conversions here
-        if (tableFieldTypeInfo.getCategory() != Category.PRIMITIVE) {
+        if (tableFieldTypeInfo.getCategory() != Category.PRIMITIVE.toMetastoreTypeCategory()) {
           // cannot convert to complex types
           column = null;
         } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 14217e3..b20028f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -935,11 +935,11 @@ public class TypeCheckProcFactory {
         TypeInfo objectTypeInfo = object.getTypeInfo();
 
         // Allow accessing a field of list element structs directly from a list
-        boolean isList = (object.getTypeInfo().getCategory() == ObjectInspector.Category.LIST);
+        boolean isList = (object.getTypeInfo().getCategory() == ObjectInspector.Category.LIST.toMetastoreTypeCategory());
         if (isList) {
           objectTypeInfo = ((ListTypeInfo) objectTypeInfo).getListElementTypeInfo();
         }
-        if (objectTypeInfo.getCategory() != Category.STRUCT) {
+        if (objectTypeInfo.getCategory() != Category.STRUCT.toMetastoreTypeCategory()) {
           throw new SemanticException(ErrorMsg.INVALID_DOT.getMsg(expr));
         }
         TypeInfo t = ((StructTypeInfo) objectTypeInfo).getStructFieldTypeInfo(fieldNameString);
@@ -958,7 +958,7 @@ public class TypeCheckProcFactory {
         // Check whether this is a list or a map
         TypeInfo myt = children.get(0).getTypeInfo();
 
-        if (myt.getCategory() == Category.LIST) {
+        if (myt.getCategory() == Category.LIST.toMetastoreTypeCategory()) {
           // Only allow integer index for now
           if (!TypeInfoUtils.implicitConvertible(children.get(1).getTypeInfo(),
               TypeInfoFactory.intTypeInfo)) {
@@ -969,7 +969,7 @@ public class TypeCheckProcFactory {
           // Calculate TypeInfo
           TypeInfo t = ((ListTypeInfo) myt).getListElementTypeInfo();
           desc = new ExprNodeGenericFuncDesc(t, FunctionRegistry.getGenericUDFForIndex(), children);
-        } else if (myt.getCategory() == Category.MAP) {
+        } else if (myt.getCategory() == Category.MAP.toMetastoreTypeCategory()) {
           if (!TypeInfoUtils.implicitConvertible(children.get(1).getTypeInfo(),
               ((MapTypeInfo) myt).getMapKeyTypeInfo())) {
             throw new SemanticException(ErrorMsg.INVALID_MAPINDEX_TYPE

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java
index 73f449f..6d02542 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java
@@ -122,9 +122,9 @@ public class ExprNodeConstantDesc extends ExprNodeDesc implements Serializable {
 
   @Override
   public String getExprString() {
-    if (typeInfo.getCategory() == Category.PRIMITIVE) {
+    if (typeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
       return getFormatted(typeInfo, value);
-    } else if (typeInfo.getCategory() == Category.STRUCT) {
+    } else if (typeInfo.getCategory() == Category.STRUCT.toMetastoreTypeCategory()) {
       StringBuilder sb = new StringBuilder();
       sb.append("const struct(");
       List<?> items = (List<?>) getWritableObjectInspector().getWritableConstantValue();

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java
index 0cfa646..3a4efd9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java
@@ -82,8 +82,8 @@ public class VectorPartitionConversion  {
   public static boolean isImplicitVectorColumnConversion(TypeInfo fromTypeInfo,
       TypeInfo toTypeInfo) {
 
-    if (fromTypeInfo.getCategory() == Category.PRIMITIVE &&
-        toTypeInfo.getCategory() == Category.PRIMITIVE) {
+    if (fromTypeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory() &&
+        toTypeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
 
       PrimitiveCategory fromPrimitiveCategory =
           ((PrimitiveTypeInfo) fromTypeInfo).getPrimitiveCategory();

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index cef87f5..69c3371 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -1952,7 +1952,7 @@ public class StatsUtils {
   public static boolean hasDiscreteRange(ColStatistics colStat) {
     if (colStat.getRange() != null) {
       TypeInfo colType = TypeInfoUtils.getTypeInfoFromTypeString(colStat.getColumnType());
-      if (colType.getCategory() == Category.PRIMITIVE) {
+      if (colType.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
         PrimitiveTypeInfo pti = (PrimitiveTypeInfo) colType;
         switch (pti.getPrimitiveCategory()) {
           case BOOLEAN:

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
index d170d86..7950840 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
@@ -76,7 +76,7 @@ public class GenericUDAFAverage extends AbstractGenericUDAFResolver {
           "Exactly one argument is expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
               + parameters[0].getTypeName() + " is passed.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBinarySetFunctions.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBinarySetFunctions.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBinarySetFunctions.java
index 397ec92..063a5c0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBinarySetFunctions.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBinarySetFunctions.java
@@ -412,12 +412,12 @@ public class GenericUDAFBinarySetFunctions extends AbstractGenericUDAFResolver {
           "Exactly two arguments are expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " is passed.");
     }
 
-    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(1, "Only primitive type arguments are accepted but "
           + parameters[1].getTypeName() + " is passed.");
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
index 2267589..59ef5d1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
@@ -71,7 +71,7 @@ public class GenericUDAFComputeStats extends AbstractGenericUDAFResolver {
           "Exactly 2 (col + hll) or 3 (col + fm + #bitvectors) arguments are expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " is passed.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFContextNGrams.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFContextNGrams.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFContextNGrams.java
index 2e7a559..9ac3ea3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFContextNGrams.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFContextNGrams.java
@@ -69,7 +69,7 @@ public class GenericUDAFContextNGrams implements GenericUDAFResolver {
     // Validate the first parameter, which is the expression to compute over. This should be an
     // array of strings type, or an array of arrays of strings.
     PrimitiveTypeInfo pti;
-    if (parameters[0].getCategory() != ObjectInspector.Category.LIST) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.LIST.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only list type arguments are accepted but "
           + parameters[0].getTypeName() + " was passed as parameter 1.");
@@ -100,9 +100,9 @@ public class GenericUDAFContextNGrams implements GenericUDAFResolver {
     }
 
     // Validate the second parameter, which should be an array of strings
-    if(parameters[1].getCategory() != ObjectInspector.Category.LIST ||
+    if(parameters[1].getCategory() != ObjectInspector.Category.LIST.toMetastoreTypeCategory() ||
        ((ListTypeInfo) parameters[1]).getListElementTypeInfo().getCategory() !=
-         ObjectInspector.Category.PRIMITIVE) {
+         ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(1, "Only arrays of strings are accepted but "
           + parameters[1].getTypeName() + " was passed as parameter 2.");
     }
@@ -113,7 +113,7 @@ public class GenericUDAFContextNGrams implements GenericUDAFResolver {
     }
 
     // Validate the third parameter, which should be an integer to represent 'k'
-    if(parameters[2].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if(parameters[2].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(2, "Only integers are accepted but "
             + parameters[2].getTypeName() + " was passed as parameter 3.");
     }
@@ -133,7 +133,7 @@ public class GenericUDAFContextNGrams implements GenericUDAFResolver {
     // If the fourth parameter -- precision factor 'pf' -- has been specified, make sure it's
     // an integer.
     if(parameters.length == 4) {
-      if(parameters[3].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+      if(parameters[3].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
         throw new UDFArgumentTypeException(3, "Only integers are accepted but "
             + parameters[3].getTypeName() + " was passed as parameter 4.");
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java
index d1517ab..8140724 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java
@@ -82,13 +82,13 @@ public class GenericUDAFCorrelation extends AbstractGenericUDAFResolver {
           "Exactly two arguments are expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " is passed.");
     }
 
-    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
         throw new UDFArgumentTypeException(1,
             "Only primitive type arguments are accepted but "
             + parameters[1].getTypeName() + " is passed.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
index 8b088f8..fb9fc5b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
@@ -69,13 +69,13 @@ public class GenericUDAFCovariance extends AbstractGenericUDAFResolver {
           "Exactly two arguments are expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " is passed.");
     }
 
-    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
         throw new UDFArgumentTypeException(1,
             "Only primitive type arguments are accepted but "
             + parameters[1].getTypeName() + " is passed.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovarianceSample.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovarianceSample.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovarianceSample.java
index 0dc3918..db8a066 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovarianceSample.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovarianceSample.java
@@ -50,13 +50,13 @@ public class GenericUDAFCovarianceSample extends GenericUDAFCovariance {
           "Exactly two arguments are expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " is passed.");
     }
 
-    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
         throw new UDFArgumentTypeException(1,
             "Only primitive type arguments are accepted but "
             + parameters[1].getTypeName() + " is passed.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
index 4910fe1..a8f369f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
@@ -69,7 +69,7 @@ public class GenericUDAFHistogramNumeric extends AbstractGenericUDAFResolver {
     }
 
     // validate the first parameter, which is the expression to compute over
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " was passed as parameter 1.");
@@ -94,7 +94,7 @@ public class GenericUDAFHistogramNumeric extends AbstractGenericUDAFResolver {
     }
 
     // validate the second parameter, which is the number of histogram bins
-    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(1,
           "Only primitive type arguments are accepted but "
           + parameters[1].getTypeName() + " was passed as parameter 2.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStd.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStd.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStd.java
index 79b519c..478db4b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStd.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStd.java
@@ -44,7 +44,7 @@ public class GenericUDAFStd extends GenericUDAFVariance {
           "Exactly one argument is expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " is passed.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStdSample.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStdSample.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStdSample.java
index c68d77c..7e33892 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStdSample.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFStdSample.java
@@ -46,7 +46,7 @@ public class GenericUDAFStdSample extends GenericUDAFVariance {
           "Exactly one argument is expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " is passed.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
index 1439b64..f755c52 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
@@ -69,7 +69,7 @@ public class GenericUDAFSum extends AbstractGenericUDAFResolver {
           "Exactly one argument is expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
               + parameters[0].getTypeName() + " is passed.");
@@ -111,7 +111,7 @@ public class GenericUDAFSum extends AbstractGenericUDAFResolver {
   }
 
   public static PrimitiveObjectInspector.PrimitiveCategory getReturnType(TypeInfo type) {
-    if (type.getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (type.getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       return null;
     }
     switch (((PrimitiveTypeInfo) type).getPrimitiveCategory()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSumEmptyIsZero.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSumEmptyIsZero.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSumEmptyIsZero.java
index 01c933c..725b361 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSumEmptyIsZero.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSumEmptyIsZero.java
@@ -35,7 +35,7 @@ public class GenericUDAFSumEmptyIsZero extends GenericUDAFSum {
           "Exactly one argument is expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
               + parameters[0].getTypeName() + " is passed.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
index c9fb3df..a89d2a8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
@@ -158,7 +158,7 @@ public class GenericUDAFVariance extends AbstractGenericUDAFResolver {
           "Exactly one argument is expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " is passed.");


[3/5] hive git commit: HIVE-17580 : Remove dependency of get_fields_with_environment_context API to serde (Vihang Karajgaonkar, reviewed by Alan Gates)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
index f3b19f0..f47aeb9 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
@@ -25,14 +25,11 @@ import java.lang.reflect.Type;
 import java.util.ArrayList;
 import java.util.EnumMap;
 import java.util.HashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -47,7 +44,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector;
 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.PrimitiveGrouping;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoParser.PrimitiveParts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -240,15 +237,6 @@ public final class TypeInfoUtils {
     }
   }
 
-  public static String getBaseName(String typeName) {
-    int idx = typeName.indexOf('(');
-    if (idx == -1) {
-      return typeName;
-    } else {
-      return typeName.substring(0, idx);
-    }
-  }
-
   /**
    * returns true if both TypeInfos are of primitive type, and the primitive category matches.
    * @param ti1
@@ -256,7 +244,8 @@ public final class TypeInfoUtils {
    * @return
    */
   public static boolean doPrimitiveCategoriesMatch(TypeInfo ti1, TypeInfo ti2) {
-    if (ti1.getCategory() == Category.PRIMITIVE && ti2.getCategory() == Category.PRIMITIVE) {
+    if (ti1.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()
+        && ti2.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       if (((PrimitiveTypeInfo)ti1).getPrimitiveCategory()
           == ((PrimitiveTypeInfo)ti2).getPrimitiveCategory()) {
         return true;
@@ -266,328 +255,10 @@ public final class TypeInfoUtils {
   }
 
   /**
-   * 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();
-   */
-  private static 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) {
-      this.typeInfoString = typeInfoString;
-      typeInfoTokens = tokenize(typeInfoString);
-    }
-
-    private final String typeInfoString;
-    private final ArrayList<Token> typeInfoTokens;
-    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 (!serdeConstants.LIST_TYPE_NAME.equals(t.text)
-            && !serdeConstants.MAP_TYPE_NAME.equals(t.text)
-            && !serdeConstants.STRUCT_TYPE_NAME.equals(t.text)
-            && !serdeConstants.UNION_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 + "' 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?
-      PrimitiveTypeEntry typeEntry =
-          PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(t.text);
-      if (typeEntry != null && typeEntry.primitiveCategory != PrimitiveCategory.UNKNOWN ) {
-        String[] params = parseParams();
-        switch (typeEntry.primitiveCategory) {
-        case CHAR:
-        case VARCHAR:
-          if (params == null || params.length == 0) {
-            throw new IllegalArgumentException(typeEntry.typeName
-                + " type is specified without length: " + typeInfoString);
-          }
-
-          int length = 1;
-          if (params.length == 1) {
-            length = Integer.parseInt(params[0]);
-            if (typeEntry.primitiveCategory == PrimitiveCategory.VARCHAR) {
-              BaseCharUtils.validateVarcharParameter(length);
-              return TypeInfoFactory.getVarcharTypeInfo(length);
-            } else {
-              BaseCharUtils.validateCharParameter(length);
-              return TypeInfoFactory.getCharTypeInfo(length);
-            }
-          } else if (params.length > 1) {
-            throw new IllegalArgumentException(
-                "Type " + typeEntry.typeName+ " only takes one parameter, but " +
-                params.length + " is seen");
-          }
-
-        case DECIMAL:
-          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]);
-            HiveDecimalUtils.validateParameter(precision, scale);
-          } else if (params.length == 2) {
-            // New metadata always have two parameters.
-            precision = Integer.parseInt(params[0]);
-            scale = Integer.parseInt(params[1]);
-            HiveDecimalUtils.validateParameter(precision, scale);
-          } else if (params.length > 2) {
-            throw new IllegalArgumentException("Type decimal only takes two parameter, but " +
-                params.length + " is seen");
-          }
-          return TypeInfoFactory.getDecimalTypeInfo(precision, scale);
-
-        default:
-          return TypeInfoFactory.getPrimitiveTypeInfo(typeEntry.typeName);
-        }
-      }
-
-      // Is this a list type?
-      if (serdeConstants.LIST_TYPE_NAME.equals(t.text)) {
-        expect("<");
-        TypeInfo listElementType = parseType();
-        expect(">");
-        return TypeInfoFactory.getListTypeInfo(listElementType);
-      }
-
-      // Is this a map type?
-      if (serdeConstants.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 (serdeConstants.STRUCT_TYPE_NAME.equals(t.text)) {
-        ArrayList<String> fieldNames = new ArrayList<String>();
-        ArrayList<TypeInfo> fieldTypeInfos = new ArrayList<TypeInfo>();
-        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 (serdeConstants.UNION_TYPE_NAME.equals(t.text)) {
-        List<TypeInfo> objectTypeInfos = new ArrayList<TypeInfo>();
-        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;
-  }
-
-  /**
    * Make some of the TypeInfo parsing available as a utility.
    */
   public static PrimitiveParts parsePrimitiveParts(String typeInfoString) {
-    TypeInfoParser parser = new TypeInfoParser(typeInfoString);
+    TypeInfoParser parser = new TypeInfoParser(typeInfoString, TypeInfoFactory.getInstance());
     return parser.parsePrimitiveParts();
   }
 
@@ -844,7 +515,7 @@ public final class TypeInfoUtils {
   }
 
   public static ArrayList<TypeInfo> getTypeInfosFromTypeString(String typeString) {
-    TypeInfoParser parser = new TypeInfoParser(typeString);
+    TypeInfoParser parser = new TypeInfoParser(typeString, TypeInfoFactory.getInstance());
     return parser.parseTypeInfos();
   }
 
@@ -861,7 +532,7 @@ public final class TypeInfoUtils {
   }
 
   public static TypeInfo getTypeInfoFromTypeString(String typeString) {
-    TypeInfoParser parser = new TypeInfoParser(typeString);
+    TypeInfoParser parser = new TypeInfoParser(typeString, TypeInfoFactory.getInstance());
     return parser.parseTypeInfos().get(0);
   }
 
@@ -972,7 +643,8 @@ public final class TypeInfoUtils {
     // Reimplemented to use PrimitiveCategory rather than TypeInfo, because
     // 2 TypeInfos from the same qualified type (varchar, decimal) should still be
     // seen as equivalent.
-    if (from.getCategory() == Category.PRIMITIVE && to.getCategory() == Category.PRIMITIVE) {
+    if (from.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()
+        && to.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
       return implicitConvertible(
           ((PrimitiveTypeInfo) from).getPrimitiveCategory(),
           ((PrimitiveTypeInfo) to).getPrimitiveCategory());

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/UnionTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/UnionTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/UnionTypeInfo.java
deleted file mode 100644
index 842997c..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/UnionTypeInfo.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.List;
-
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-
-/**
- * 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(serdeConstants.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 Category getCategory() {
-    return Category.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();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/VarcharTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/VarcharTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/VarcharTypeInfo.java
index edf12a2..a6c248a 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/VarcharTypeInfo.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/VarcharTypeInfo.java
@@ -51,7 +51,6 @@ public class VarcharTypeInfo extends BaseCharTypeInfo {
 
     return this.getLength() == pti.getLength();
   }
-
   /**
    * Generate the hashCode for this TypeInfo.
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java b/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java
index 749d8ac..655d768 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java
@@ -435,7 +435,7 @@ public class SerdeRandomRowSource {
       }
       
       typeInfos[c] = typeInfo;
-      final Category category = typeInfo.getCategory();
+      final Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
       categories[c] = category;
       ObjectInspector objectInspector = getObjectInspector(typeInfo);
       switch (category) {
@@ -640,7 +640,7 @@ public class SerdeRandomRowSource {
                 elementObjectInspector);
         boolean isStringFamily = false;
         PrimitiveCategory primitiveCategory = null;
-        if (elementTypeInfo.getCategory() == Category.PRIMITIVE) {
+        if (elementTypeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
           primitiveCategory = ((PrimitiveTypeInfo) elementTypeInfo).getPrimitiveCategory();
           if (primitiveCategory == PrimitiveCategory.STRING ||
               primitiveCategory == PrimitiveCategory.BINARY ||

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroObjectInspectorGenerator.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroObjectInspectorGenerator.java b/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroObjectInspectorGenerator.java
index 3736a1f..6bcde38 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroObjectInspectorGenerator.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroObjectInspectorGenerator.java
@@ -27,6 +27,7 @@ import java.util.List;
 import org.apache.avro.Schema;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
@@ -403,7 +404,7 @@ public class TestAvroObjectInspectorGenerator {
     // Column types
     assertEquals(1, aoig.getColumnTypes().size());
     TypeInfo typeInfo = aoig.getColumnTypes().get(0);
-    assertEquals(ObjectInspector.Category.MAP, typeInfo.getCategory());
+    assertEquals(ObjectInspector.Category.MAP, Category.fromMetastoreTypeCategory(typeInfo.getCategory()));
     assertTrue(typeInfo instanceof MapTypeInfo);
     MapTypeInfo mapTypeInfo = (MapTypeInfo)typeInfo;
 
@@ -423,7 +424,7 @@ public class TestAvroObjectInspectorGenerator {
     // Column types
     assertEquals(1, aoig.getColumnTypes().size());
     TypeInfo typeInfo = aoig.getColumnTypes().get(0);
-    assertEquals(ObjectInspector.Category.LIST, typeInfo.getCategory());
+    assertEquals(ObjectInspector.Category.LIST, Category.fromMetastoreTypeCategory(typeInfo.getCategory()));
     assertTrue(typeInfo instanceof ListTypeInfo);
     ListTypeInfo listTypeInfo = (ListTypeInfo)typeInfo;
 
@@ -442,7 +443,7 @@ public class TestAvroObjectInspectorGenerator {
     // Column types
     assertEquals(1, aoig.getColumnTypes().size());
     TypeInfo typeInfo = aoig.getColumnTypes().get(0);
-    assertEquals(ObjectInspector.Category.STRUCT, typeInfo.getCategory());
+    assertEquals(ObjectInspector.Category.STRUCT, Category.fromMetastoreTypeCategory(typeInfo.getCategory()));
     assertTrue(typeInfo instanceof StructTypeInfo);
     StructTypeInfo structTypeInfo = (StructTypeInfo)typeInfo;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java b/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
deleted file mode 100644
index cb7c6ed..0000000
--- a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * 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 java.util.Set;
-import org.junit.Test;
-
-import static org.junit.Assert.assertSame;
-
-public class TestInstanceCache {
-  private static class Foo {
-
-    private int value = 42;
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) return true;
-      if (o == null || getClass() != o.getClass()) return false;
-
-      Foo foo = (Foo) o;
-
-      return value == foo.value;
-
-    }
-
-    @Override
-    public int hashCode() {
-      return value;
-    }
-  }
-
-  private static class Wrapper<T> {
-    public final T wrapped;
-
-    private Wrapper(T wrapped) {
-      this.wrapped = wrapped;
-    }
-  }
-
-  @Test
-  public void instanceCachesOnlyCreateOneInstance() throws AvroSerdeException {
-    InstanceCache<Foo, Wrapper<Foo>> ic = new InstanceCache<Foo, Wrapper<Foo>>() {
-      @Override
-      protected Wrapper makeInstance(Foo hv,
-                                     Set<Foo> seenSchemas) {
-        return new Wrapper(hv);
-      }
-    };
-    Foo f1 = new Foo();
-
-    Wrapper fc = ic.retrieve(f1, null);
-    assertSame(f1, fc.wrapped); // Our original foo should be in the wrapper
-
-    Foo f2 = new Foo(); // Different instance, same value
-
-    Wrapper fc2 = ic.retrieve(f2, null);
-    assertSame(fc2,fc); // Since equiv f, should get back first container
-    assertSame(fc2.wrapped, f1);
-  }
-
-  @Test
-  public void instanceCacheReturnsCorrectInstances() throws AvroSerdeException {
-    InstanceCache<String, Wrapper<String>> ic = new InstanceCache<String, Wrapper<String>>() {
-      @Override
-      protected Wrapper<String> makeInstance(
-          String hv, Set<String> seenSchemas) {
-        return new Wrapper<String>(hv);
-      }
-    };
-
-    Wrapper<String> one = ic.retrieve("one", null);
-    Wrapper<String> two = ic.retrieve("two", null);
-
-    Wrapper<String> anotherOne = ic.retrieve("one", null);
-    assertSame(one, anotherOne);
-
-    Wrapper<String> anotherTwo = ic.retrieve("two", null);
-    assertSame(two, anotherTwo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestSchemaToTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestSchemaToTypeInfo.java b/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestSchemaToTypeInfo.java
index af258c7..7a663c0 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestSchemaToTypeInfo.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestSchemaToTypeInfo.java
@@ -33,9 +33,9 @@ public class TestSchemaToTypeInfo {
 
   @Test
   public void testDisallowRecursiveSchema()
-      throws AvroSerdeException {
+      throws Exception {
 
-    expect.expect(AvroSerdeException.class);
+    expect.expect(Exception.class);
     expect.expectMessage("Recursive schemas are not supported");
 
     final String schemaString = "{\n"
@@ -49,6 +49,7 @@ public class TestSchemaToTypeInfo {
         + "  } ]\n"
         + "}";
 
-    List<TypeInfo> types = SchemaToTypeInfo.generateColumnTypes(new Schema.Parser().parse(schemaString));
+    List<TypeInfo> types = SchemaToHiveTypeInfo.getInstance()
+        .generateColumnTypes(new Schema.Parser().parse(schemaString));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
index 62741d3..f1e3228 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
@@ -313,7 +313,7 @@ public class TestBinarySortableFast extends TestCase {
 
   private void verifyRead(BinarySortableDeserializeRead binarySortableDeserializeRead,
       TypeInfo typeInfo, Object expectedObject) throws IOException {
-    if (typeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE) {
+    if (typeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       VerifyFast.verifyDeserializeRead(binarySortableDeserializeRead, typeInfo, expectedObject);
     } else {
       Object complexFieldObj = VerifyFast.deserializeReadComplexType(binarySortableDeserializeRead, typeInfo);

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleFast.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleFast.java
index fbb6040..97cf220 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleFast.java
@@ -199,7 +199,7 @@ public class TestLazySimpleFast extends TestCase {
 
   private void verifyReadNull(LazySimpleDeserializeRead lazySimpleDeserializeRead,
       TypeInfo typeInfo) throws IOException {
-    if (typeInfo.getCategory() == Category.PRIMITIVE) {
+    if (typeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
       VerifyFast.verifyDeserializeRead(lazySimpleDeserializeRead, typeInfo, null);
     } else {
       Object complexFieldObj = VerifyFast.deserializeReadComplexType(lazySimpleDeserializeRead, typeInfo);
@@ -211,7 +211,7 @@ public class TestLazySimpleFast extends TestCase {
 
   private void verifyRead(LazySimpleDeserializeRead lazySimpleDeserializeRead,
       TypeInfo typeInfo, Object expectedObject) throws IOException {
-    if (typeInfo.getCategory() == Category.PRIMITIVE) {
+    if (typeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
       VerifyFast.verifyDeserializeRead(lazySimpleDeserializeRead, typeInfo, expectedObject);
     } else {
       Object complexFieldObj = VerifyFast.deserializeReadComplexType(lazySimpleDeserializeRead, typeInfo);

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
index e95c6eb..d42eda0 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
@@ -211,7 +211,7 @@ public class TestLazyBinaryFast extends TestCase {
 
   private void verifyRead(LazyBinaryDeserializeRead lazyBinaryDeserializeRead,
       TypeInfo typeInfo, Object expectedObject) throws IOException {
-    if (typeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE) {
+    if (typeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       VerifyFast.verifyDeserializeRead(lazyBinaryDeserializeRead, typeInfo, expectedObject);
     } else {
       Object complexFieldObj = VerifyFast.deserializeReadComplexType(lazyBinaryDeserializeRead, typeInfo);

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestStandardObjectInspectors.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestStandardObjectInspectors.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestStandardObjectInspectors.java
index 17b844c..269eb98 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestStandardObjectInspectors.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestStandardObjectInspectors.java
@@ -458,7 +458,7 @@ public class TestStandardObjectInspectors extends TestCase {
       assertEquals(unionTypeName.toString(), uoi1.getTypeName());
       // TypeInfo
       TypeInfo typeInfo1 = TypeInfoUtils.getTypeInfoFromObjectInspector(uoi1);
-      assertEquals(Category.UNION, typeInfo1.getCategory());
+      assertEquals(Category.UNION, Category.fromMetastoreTypeCategory(typeInfo1.getCategory()));
       assertEquals(UnionTypeInfo.class.getName(), typeInfo1.getClass().getName());
       assertEquals(typeInfo1.getTypeName(), uoi1.getTypeName());
       assertEquals(typeInfo1,

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AvroStorageSchemaReader.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AvroStorageSchemaReader.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AvroStorageSchemaReader.java
new file mode 100644
index 0000000..c6c3029
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AvroStorageSchemaReader.java
@@ -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.metastore;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.utils.AvroSchemaUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+
+public class AvroStorageSchemaReader implements StorageSchemaReader {
+  private static final Logger LOG = LoggerFactory.getLogger(AvroStorageSchemaReader.class);
+
+  @Override
+  public List<FieldSchema> readSchema(Table tbl, EnvironmentContext envContext,
+      Configuration conf) throws MetaException {
+    Properties tblMetadataProperties = MetaStoreUtils.getTableMetadata(tbl);
+    try {
+      return AvroSchemaUtils.getFieldsFromAvroSchema(conf, tblMetadataProperties);
+    } catch (Exception e) {
+      LOG.warn("Received IOException while reading avro schema for table " + tbl.getTableName(), e);
+      throw new MetaException(e.getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
index d5dea4d..ab8590e 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
@@ -72,6 +72,8 @@ public class ColumnType {
 
   public static final String TIMESTAMPTZ_TYPE_NAME = "timestamp with time zone";
 
+  public static final String TIMESTAMPLOCALTZ_TYPE_NAME = "timestamp with local time zone";
+
   public static final String LIST_TYPE_NAME = "array";
 
   public static final String MAP_TYPE_NAME = "map";
@@ -105,7 +107,8 @@ public class ColumnType {
     INTERVAL_DAY_TIME_TYPE_NAME,
     DECIMAL_TYPE_NAME,
     BINARY_TYPE_NAME,
-    TIMESTAMPTZ_TYPE_NAME);
+    TIMESTAMPTZ_TYPE_NAME,
+    TIMESTAMPLOCALTZ_TYPE_NAME);
 
   public static final Set<String> StringTypes = StringUtils.asSet(
       STRING_TYPE_NAME,

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultStorageSchemaReader.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultStorageSchemaReader.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultStorageSchemaReader.java
index 1dbfa42..65c2af6 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultStorageSchemaReader.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultStorageSchemaReader.java
@@ -22,17 +22,121 @@ import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.utils.AvroSchemaUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.utils.StorageSchemaUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
+import java.util.Properties;
+
+import static org.apache.hadoop.hive.metastore.utils.AvroSchemaUtils.LIST_COLUMN_COMMENTS;
 
 /**
  * Default StorageSchemaReader.  This just throws as the metastore currently doesn't know how to
  * read schemas from storage.
  */
 public class DefaultStorageSchemaReader implements StorageSchemaReader {
+  private final static Logger LOG = LoggerFactory.getLogger(DefaultStorageSchemaReader.class);
+
+  private static final String AVRO_SERIALIZATION_LIB =
+      "org.apache.hadoop.hive.serde2.avro.AvroSerDe";
+
   @Override
   public List<FieldSchema> readSchema(Table tbl, EnvironmentContext envContext,
-                                      Configuration conf) throws MetaException {
-    throw new UnsupportedOperationException("Storage schema reading not supported");
+      Configuration conf) throws MetaException {
+    String serializationLib = tbl.getSd().getSerdeInfo().getSerializationLib();
+    if (null == serializationLib || MetastoreConf
+        .getStringCollection(conf, MetastoreConf.ConfVars.SERDES_USING_METASTORE_FOR_SCHEMA)
+        .contains(serializationLib)) {
+      //safety check to make sure we should be using storage schema reader for this table
+      throw new MetaException(
+          "Invalid usage of default storage schema reader for table " + tbl.getTableName()
+              + " with storage descriptor " + tbl.getSd().getSerdeInfo().getSerializationLib());
+    }
+    Properties tblMetadataProperties = MetaStoreUtils.getTableMetadata(tbl);
+    if(AVRO_SERIALIZATION_LIB.equals(serializationLib)) {
+      //in case of avro table use AvroStorageSchemaReader utils
+      try {
+        return AvroSchemaUtils.getFieldsFromAvroSchema(conf, tblMetadataProperties);
+      } catch (Exception e) {
+        LOG.warn("Exception received while reading avro schema for table " + tbl.getTableName(), e);
+        throw new MetaException(e.getMessage());
+      }
+    } else {
+      return getFieldSchemasFromTableMetadata(tblMetadataProperties);
+    }
+  }
+
+  /**
+   * This method implements a generic way to get the FieldSchemas from the table metadata
+   * properties like column names and column types. Most of the serdes have the same implemention
+   * in their initialize method
+   * //TODO refactor the common code from the serdes and move it to serde-api so that there is no
+   * //duplicate code
+   *
+   * @return list of FieldSchema objects
+   */
+  public static List<FieldSchema> getFieldSchemasFromTableMetadata(
+      Properties tblMetadataProperties) {
+    List<String> columnNames = null;
+    List<TypeInfo> columnTypes = null;
+    // Get column names and types
+    String columnNameProperty = tblMetadataProperties.getProperty( ColumnType.LIST_COLUMNS);
+    String columnTypeProperty = tblMetadataProperties.getProperty( ColumnType.LIST_COLUMN_TYPES);
+    final String columnNameDelimiter = tblMetadataProperties
+        .containsKey( ColumnType.COLUMN_NAME_DELIMITER) ? tblMetadataProperties
+        .getProperty( ColumnType.COLUMN_NAME_DELIMITER) : String
+        .valueOf(StorageSchemaUtils.COMMA);
+    // all table column names
+    if (columnNameProperty.isEmpty()) {
+      columnNames = Collections.emptyList();
+    } else {
+      columnNames = Arrays.asList(columnNameProperty.split(columnNameDelimiter));
+    }
+
+    // all column types
+    if (columnTypeProperty.isEmpty()) {
+      columnTypes = Collections.emptyList();
+    } else {
+      columnTypes = StorageSchemaUtils.getTypeInfosFromTypeString(columnTypeProperty);
+    }
+
+    final String columnCommentProperty =
+        tblMetadataProperties.getProperty(LIST_COLUMN_COMMENTS, "");
+    List<String> columnComments = null;
+    if (columnCommentProperty == null || columnCommentProperty.isEmpty()) {
+      columnComments = new ArrayList<>(0);
+    } else {
+      columnComments = Arrays.asList(
+          columnCommentProperty.split(String.valueOf(ColumnType.COLUMN_COMMENTS_DELIMITER)));
+    }
+    LOG.debug("columns: {}, {}", columnNameProperty, columnNames);
+    LOG.debug("types: {}, {} ", columnTypeProperty, columnTypes);
+    LOG.debug("comments: {} ", columnCommentProperty);
+    return getFieldSchemaFromColumnInfo(columnNames, columnTypes, columnComments);
+  }
+
+  private static List<FieldSchema> getFieldSchemaFromColumnInfo(List<String> columnNames,
+      List<TypeInfo> columnTypes, List<String> columnComments) {
+    int len = columnNames.size();
+    List<FieldSchema> fieldSchemas = new ArrayList<>(len);
+    for (int i = 0; i < len; i++) {
+      FieldSchema fieldSchema = new FieldSchema();
+      fieldSchema.setName(columnNames.get(i));
+      //In case of complex types getTypeName() will recusively go into typeName
+      //of individual fields when the ColumnType was constructed
+      //in SchemaToTypeInfo.generateColumnTypes in the constructor
+      fieldSchema.setType(columnTypes.get(i).getTypeName());
+      fieldSchema.setComment(StorageSchemaUtils.determineFieldComment(columnComments.get(i)));
+    }
+    return fieldSchemas;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StorageSchemaReader.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StorageSchemaReader.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StorageSchemaReader.java
index 6251e23..009c929 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StorageSchemaReader.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StorageSchemaReader.java
@@ -32,7 +32,7 @@ import java.util.List;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-interface StorageSchemaReader {
+public interface StorageSchemaReader {
   /**
    * Read the schema from the storage representation of the table.
    * @param tbl metastore table object

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/AvroFieldSchemaGenerator.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/AvroFieldSchemaGenerator.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/AvroFieldSchemaGenerator.java
new file mode 100644
index 0000000..b1261d8
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/AvroFieldSchemaGenerator.java
@@ -0,0 +1,97 @@
+/*
+ * 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.metastore.utils;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.serde2.avro.SchemaToMetastoreTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class AvroFieldSchemaGenerator {
+  final private List<String> columnNames;
+  final private List<TypeInfo> columnTypes;
+  final private List<String> columnComments;
+
+  public AvroFieldSchemaGenerator(Schema schema) throws Exception {
+    verifySchemaIsARecord(schema);
+
+    this.columnNames = generateColumnNames(schema);
+    this.columnTypes = SchemaToMetastoreTypeInfo.getInstance().generateColumnTypes(schema);
+    this.columnComments = generateColumnComments(schema);
+    assert columnNames.size() == columnTypes.size();
+  }
+
+  private static void verifySchemaIsARecord(Schema schema) throws Exception {
+    if(!schema.getType().equals(Schema.Type.RECORD)) {
+      throw new Exception("Schema for table must be of type RECORD. " +
+          "Received type: " + schema.getType());
+    }
+  }
+
+  private static List<String> generateColumnNames(Schema schema) {
+    List<Schema.Field> fields = schema.getFields();
+    List<String> fieldsList = new ArrayList<String>(fields.size());
+
+    for (Schema.Field field : fields) {
+      fieldsList.add(field.name());
+    }
+
+    return fieldsList;
+  }
+
+  private static List<String> generateColumnComments(Schema schema) {
+    List<Schema.Field> fields = schema.getFields();
+    List<String> fieldComments = new ArrayList<String>(fields.size());
+
+    for (Schema.Field field : fields) {
+      String fieldComment = field.doc() == null ? "" : field.doc();
+      fieldComments.add(fieldComment);
+    }
+
+    return fieldComments;
+  }
+
+  public List<FieldSchema> getFieldSchemas() throws Exception {
+    int len = columnNames.size();
+    List<FieldSchema> fieldSchemas = new ArrayList<>(len);
+    for(int i = 0; i<len; i++) {
+      FieldSchema fieldSchema = new FieldSchema();
+      fieldSchema.setName(columnNames.get(i));
+       TypeInfo columnType = columnTypes.get(i);
+      if(!AvroSchemaUtils.supportedCategories(columnType)) {
+        throw new Exception("Don't yet support this type: " + columnType);
+      }
+      //In case of complex types getTypeName() will recusively go into typeName
+      //of individual fields when the ColumnType was constructed
+      //in SchemaToTypeInfo.generateColumnTypes in the constructor
+      fieldSchema.setType(columnTypes.get(i).getTypeName());
+      fieldSchema.setComment(StorageSchemaUtils.determineFieldComment(columnComments.get(i)));
+      fieldSchemas.add(fieldSchema);
+    }
+    return fieldSchemas;
+  }
+
+  private static final String FROM_SERIALIZER = "from deserializer";
+
+  private static String determineFieldComment(String comment) {
+    return (comment == null) ? FROM_SERIALIZER : comment;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/AvroSchemaUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/AvroSchemaUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/AvroSchemaUtils.java
new file mode 100644
index 0000000..d7bcd15
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/AvroSchemaUtils.java
@@ -0,0 +1,366 @@
+/*
+ * 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.metastore.utils;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.ColumnType;
+import org.apache.hadoop.hive.serde2.typeinfo.MetastoreTypeCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.MetastoreTypeInfoFactory;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.serde2.avro.AvroSerDeConstants;
+import org.apache.hadoop.hive.serde2.avro.SchemaResolutionProblem;
+import org.apache.hadoop.hive.serde2.avro.TypeInfoToSchema;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.*;
+
+/*
+ * Many of the util methods are copied from AvroSerDeUtils from Hive
+ */
+public class AvroSchemaUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(AvroSchemaUtils.class);
+  /**
+   * Enum container for all avro table properties.
+   * If introducing a new avro-specific table property,
+   * add it here. Putting them in an enum rather than separate strings
+   * allows them to be programmatically grouped and referenced together.
+   */
+  public static enum AvroTableProperties {
+    SCHEMA_LITERAL("avro.schema.literal"),
+    SCHEMA_URL("avro.schema.url"),
+    SCHEMA_NAMESPACE("avro.schema.namespace"),
+    SCHEMA_NAME("avro.schema.name"),
+    SCHEMA_DOC("avro.schema.doc"),
+    AVRO_SERDE_SCHEMA("avro.serde.schema"),
+    SCHEMA_RETRIEVER("avro.schema.retriever");
+
+    private final String propName;
+
+    AvroTableProperties(String propName) {
+      this.propName = propName;
+    }
+
+    public String getPropName(){
+      return this.propName;
+    }
+  }
+
+  // Following parameters slated for removal, prefer usage of enum above, that allows programmatic access.
+  @Deprecated public static final String SCHEMA_LITERAL = "avro.schema.literal";
+  @Deprecated public static final String SCHEMA_URL = "avro.schema.url";
+  @Deprecated public static final String SCHEMA_NAMESPACE = "avro.schema.namespace";
+  @Deprecated public static final String SCHEMA_NAME = "avro.schema.name";
+  @Deprecated public static final String SCHEMA_DOC = "avro.schema.doc";
+  @Deprecated public static final String AVRO_SERDE_SCHEMA = AvroTableProperties.AVRO_SERDE_SCHEMA.getPropName();
+  @Deprecated public static final String SCHEMA_RETRIEVER = AvroTableProperties.SCHEMA_RETRIEVER.getPropName();
+
+  public static final String SCHEMA_NONE = "none";
+  public static final String EXCEPTION_MESSAGE = "Neither "
+      + AvroTableProperties.SCHEMA_LITERAL.getPropName() + " nor "
+      + AvroTableProperties.SCHEMA_URL.getPropName() + " specified, can't determine table schema";
+
+  public static final String LIST_COLUMN_COMMENTS = "columns.comments";
+  public static final char COMMA = ',';
+
+  public static List<FieldSchema> getFieldsFromAvroSchema(Configuration configuration,
+      Properties properties) throws Exception {
+    // Reset member variables so we don't get in a half-constructed state
+    Schema schema = null;
+    List<String> columnNames = null;
+    List<TypeInfo> columnTypes = null;
+
+    final String columnNameProperty = properties.getProperty(ColumnType.LIST_COLUMNS);
+    final String columnTypeProperty = properties.getProperty(ColumnType.LIST_COLUMN_TYPES);
+    final String columnCommentProperty = properties.getProperty(LIST_COLUMN_COMMENTS,"");
+    final String columnNameDelimiter = properties.containsKey(ColumnType.COLUMN_NAME_DELIMITER) ? properties
+        .getProperty(ColumnType.COLUMN_NAME_DELIMITER) : String.valueOf(COMMA);
+
+    if (hasExternalSchema(properties)
+        || columnNameProperty == null || columnNameProperty.isEmpty()
+        || columnTypeProperty == null || columnTypeProperty.isEmpty()) {
+      schema = AvroSchemaUtils.determineSchemaOrThrowException(configuration, properties);
+    } else {
+      // Get column names and sort order
+      columnNames = StringUtils.intern(
+          Arrays.asList(columnNameProperty.split(columnNameDelimiter)));
+      columnTypes = new TypeInfoParser(columnTypeProperty, MetastoreTypeInfoFactory
+          .getInstance()).parseTypeInfos();
+
+      schema = getSchemaFromCols(properties, columnNames, columnTypes, columnCommentProperty);
+      properties.setProperty(AvroTableProperties.SCHEMA_LITERAL.getPropName(), schema.toString());
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Avro schema is " + schema);
+    }
+
+    if (configuration == null) {
+      LOG.debug("Configuration null, not inserting schema");
+    } else {
+      configuration.set(
+          AvroTableProperties.AVRO_SERDE_SCHEMA.getPropName(), schema.toString(false));
+    }
+    return new AvroFieldSchemaGenerator(schema).getFieldSchemas();
+  }
+
+
+  private static boolean hasExternalSchema(Properties properties) {
+    return properties.getProperty(AvroTableProperties.SCHEMA_LITERAL.getPropName()) != null
+        || properties.getProperty(AvroTableProperties.SCHEMA_URL.getPropName()) != null;
+  }
+
+  public static boolean supportedCategories(TypeInfo ti) {
+    final MetastoreTypeCategory c = ti.getCategory();
+    return c.equals(MetastoreTypeCategory.PRIMITIVE) ||
+        c.equals(MetastoreTypeCategory.MAP)          ||
+        c.equals(MetastoreTypeCategory.LIST)         ||
+        c.equals(MetastoreTypeCategory.STRUCT)       ||
+        c.equals(MetastoreTypeCategory.UNION);
+  }
+
+  /**
+   * Attempt to determine the schema via the usual means, but do not throw
+   * an exception if we fail.  Instead, signal failure via a special
+   * schema.
+   */
+  public static Schema determineSchemaOrReturnErrorSchema(Configuration conf, Properties props) {
+    try {
+      return AvroSchemaUtils.determineSchemaOrThrowException(conf, props);
+    } catch (Exception e) {
+      LOG.warn("Encountered exception determining schema. Returning signal " +
+          "schema to indicate problem", e);
+    }
+    return SchemaResolutionProblem.SIGNAL_BAD_SCHEMA;
+  }
+
+  /**
+   * Determine the schema to that's been provided for Avro serde work.
+   * @param properties containing a key pointing to the schema, one way or another
+   * @return schema to use while serdeing the avro file
+   */
+  public static Schema determineSchemaOrThrowException(Configuration conf, Properties properties)
+      throws Exception {
+    String schemaString = properties.getProperty(AvroTableProperties.SCHEMA_LITERAL.getPropName());
+    if(schemaString != null && !schemaString.equals(SCHEMA_NONE))
+      return AvroSchemaUtils.getSchemaFor(schemaString);
+
+    // Try pulling directly from URL
+    schemaString = properties.getProperty(AvroTableProperties.SCHEMA_URL.getPropName());
+    if (schemaString == null) {
+      final String columnNameProperty = properties.getProperty(ColumnType.LIST_COLUMNS);
+      final String columnTypeProperty = properties.getProperty(ColumnType.LIST_COLUMN_TYPES);
+      final String columnCommentProperty = properties.getProperty(LIST_COLUMN_COMMENTS);
+      if (columnNameProperty == null || columnNameProperty.isEmpty()
+          || columnTypeProperty == null || columnTypeProperty.isEmpty() ) {
+        throw new IOException(EXCEPTION_MESSAGE);
+      }
+      final String columnNameDelimiter = properties.containsKey(ColumnType.COLUMN_NAME_DELIMITER) ? properties
+          .getProperty(ColumnType.COLUMN_NAME_DELIMITER) : String.valueOf(COMMA);
+      // Get column names and types
+      List<String> columnNames = Arrays.asList(columnNameProperty.split(columnNameDelimiter));
+      List<TypeInfo> columnTypes =
+          new TypeInfoParser(columnTypeProperty,
+              MetastoreTypeInfoFactory.getInstance()).parseTypeInfos();
+      //TODO Why can't we directly bypass this whole logic and use ColumnTypeInfo to use
+      //AvroFieldSchemaGenerator directly?
+      Schema schema = getSchemaFromCols(properties, columnNames, columnTypes, columnCommentProperty);
+      properties.setProperty(AvroTableProperties.SCHEMA_LITERAL.getPropName(), schema.toString());
+      if (conf != null)
+        conf.set(AvroTableProperties.AVRO_SERDE_SCHEMA.getPropName(), schema.toString(false));
+      return schema;
+    } else if(schemaString.equals(SCHEMA_NONE)) {
+      throw new Exception(EXCEPTION_MESSAGE);
+    }
+
+    try {
+      Schema s = getSchemaFromFS(schemaString, conf);
+      if (s == null) {
+        //in case schema is not a file system
+        return AvroSchemaUtils.getSchemaFor(new URL(schemaString));
+      }
+      return s;
+    } catch (IOException ioe) {
+      throw new Exception("Unable to read schema from given path: " + schemaString, ioe);
+    } catch (URISyntaxException urie) {
+      throw new Exception("Unable to read schema from given path: " + schemaString, urie);
+    }
+  }
+
+  // Protected for testing and so we can pass in a conf for testing.
+  protected static Schema getSchemaFromFS(String schemaFSUrl,
+      Configuration conf) throws IOException, URISyntaxException {
+    FSDataInputStream in = null;
+    FileSystem fs = null;
+    try {
+      fs = FileSystem.get(new URI(schemaFSUrl), conf);
+    } catch (IOException ioe) {
+      //return null only if the file system in schema is not recognized
+      if (LOG.isDebugEnabled()) {
+        String msg = "Failed to open file system for uri " + schemaFSUrl + " assuming it is not a FileSystem url";
+        LOG.debug(msg, ioe);
+      }
+
+      return null;
+    }
+    try {
+      in = fs.open(new Path(schemaFSUrl));
+      Schema s = AvroSchemaUtils.getSchemaFor(in);
+      return s;
+    } finally {
+      if(in != null) in.close();
+    }
+  }
+
+  public static Schema getSchemaFor(File file) {
+    Schema.Parser parser = new Schema.Parser();
+    Schema schema;
+    try {
+      schema = parser.parse(file);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to parse Avro schema from " + file.getName(), e);
+    }
+    return schema;
+  }
+
+  public static Schema getSchemaFor(InputStream stream) {
+    Schema.Parser parser = new Schema.Parser();
+    Schema schema;
+    try {
+      schema = parser.parse(stream);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to parse Avro schema", e);
+    }
+    return schema;
+  }
+
+  public static Schema getSchemaFor(String str) {
+    Schema.Parser parser = new Schema.Parser();
+    Schema schema = parser.parse(str);
+    return schema;
+  }
+
+  public static Schema getSchemaFor(URL url) {
+    InputStream in = null;
+    try {
+      in = url.openStream();
+      return getSchemaFor(in);
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to parse Avro schema", e);
+    } finally {
+      if (in != null) {
+        try {
+          in.close();
+        } catch (IOException e) {
+          // Ignore
+        }
+      }
+    }
+  }
+
+  public static Schema getSchemaFromCols(Properties properties,
+      List<String> columnNames, List<TypeInfo> columnTypes, String columnCommentProperty) {
+    List<String> columnComments;
+    if (columnCommentProperty == null || columnCommentProperty.isEmpty()) {
+      columnComments = new ArrayList<String>();
+    } else {
+      //Comments are separated by "\0" in columnCommentProperty, see method getSchema
+      //in MetaStoreUtils where this string columns.comments is generated
+      columnComments = Arrays.asList(columnCommentProperty.split("\0"));
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("columnComments is " + columnCommentProperty);
+      }
+    }
+    if (columnNames.size() != columnTypes.size()) {
+      throw new IllegalArgumentException("getSchemaFromCols initialization failed. Number of column " +
+          "name and column type differs. columnNames = " + columnNames + ", columnTypes = " +
+          columnTypes);
+    }
+
+    final String tableName = properties.getProperty(AvroSerDeConstants.TABLE_NAME);
+    final String tableComment = properties.getProperty(AvroSerDeConstants.TABLE_COMMENT);
+    TypeInfoToSchema metastoreTypeInfoToSchema = new TypeInfoToSchema();
+    return metastoreTypeInfoToSchema.convert(columnNames, columnTypes, columnComments,
+        properties.getProperty(AvroTableProperties.SCHEMA_NAMESPACE.getPropName()),
+        properties.getProperty(AvroTableProperties.SCHEMA_NAME.getPropName(), tableName),
+        properties.getProperty(AvroTableProperties.SCHEMA_DOC.getPropName(), tableComment));
+
+  }
+
+  /**
+   * Determine if an Avro schema is of type Union[T, NULL].  Avro supports nullable
+   * types via a union of type T and null.  This is a very common use case.
+   * As such, we want to silently convert it to just T and allow the value to be null.
+   *
+   * When a Hive union type is used with AVRO, the schema type becomes
+   * Union[NULL, T1, T2, ...]. The NULL in the union should be silently removed
+   *
+   * @return true if type represents Union[T, Null], false otherwise
+   */
+  public static boolean isNullableType(Schema schema) {
+    if (!schema.getType().equals(Schema.Type.UNION)) {
+      return false;
+    }
+
+    List<Schema> itemSchemas = schema.getTypes();
+    if (itemSchemas.size() < 2) {
+      return false;
+    }
+
+    for (Schema itemSchema : itemSchemas) {
+      if (Schema.Type.NULL.equals(itemSchema.getType())) {
+        return true;
+      }
+    }
+
+    // [null, null] not allowed, so this check is ok.
+    return false;
+  }
+
+  /**
+   * In a nullable type, get the schema for the non-nullable type.  This method
+   * does no checking that the provides Schema is nullable.
+   */
+  public static Schema getOtherTypeFromNullableType(Schema schema) {
+    List<Schema> itemSchemas = new ArrayList<>();
+    for (Schema itemSchema : schema.getTypes()) {
+      if (!Schema.Type.NULL.equals(itemSchema.getType())) {
+        itemSchemas.add(itemSchema);
+      }
+    }
+
+    if (itemSchemas.size() > 1) {
+      return Schema.createUnion(itemSchemas);
+    } else {
+      return itemSchemas.get(0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/StorageSchemaUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/StorageSchemaUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/StorageSchemaUtils.java
new file mode 100644
index 0000000..5ec642f
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/StorageSchemaUtils.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.metastore.utils;
+
+import org.apache.hadoop.hive.serde2.typeinfo.MetastoreTypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoParser;
+
+import java.util.List;
+
+public class StorageSchemaUtils {
+  public static final char COMMA = ',';
+  public static List<TypeInfo> getTypeInfosFromTypeString(String columnTypeProperty) {
+    return new TypeInfoParser(columnTypeProperty, MetastoreTypeInfoFactory.getInstance())
+        .parseTypeInfos();
+  }
+
+  private static final String FROM_STORAGE_SCHEMA_READER = "generated by storage schema reader";
+  public static String determineFieldComment(String comment) {
+    return (comment == null) ? FROM_STORAGE_SCHEMA_READER : comment;
+  }
+}
\ 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/avro/AvroSerDeConstants.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/AvroSerDeConstants.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/AvroSerDeConstants.java
new file mode 100644
index 0000000..42868ea
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/AvroSerDeConstants.java
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+/**
+ * This class contains some of the constants which are specific to AvroSerDe
+ * They should always match with the constants defined in AvroSerDe.java in Hive Source code. These
+ * constants were copied as part of separating metastore from Hive.
+ */
+public class AvroSerDeConstants {
+  public static final String TABLE_NAME = "name";
+  public static final String TABLE_COMMENT = "comment";
+  public static final String LIST_COLUMN_COMMENTS = "columns.comments";
+
+  //it just so happens that the AVRO has these constants which are same as defined in ColumnType
+  //We should still keep it separate in case in future we need to separate the two
+  public static final String DECIMAL_TYPE_NAME = "decimal";
+  public static final String CHAR_TYPE_NAME = "char";
+  public static final String VARCHAR_TYPE_NAME = "varchar";
+  public static final String DATE_TYPE_NAME = "date";
+
+  public static final String AVRO_TIMESTAMP_TYPE_NAME = "timestamp-millis";
+  public static final String AVRO_PROP_LOGICAL_TYPE = "logicalType";
+  public static final String AVRO_PROP_PRECISION = "precision";
+  public static final String AVRO_PROP_SCALE = "scale";
+  public static final String AVRO_PROP_MAX_LENGTH = "maxLength";
+  public static final String AVRO_STRING_TYPE_NAME = "string";
+  public static final String AVRO_INT_TYPE_NAME = "int";
+  public static final String AVRO_LONG_TYPE_NAME = "long";
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
new file mode 100644
index 0000000..12a8ff2
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
@@ -0,0 +1,72 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Cache for objects whose creation only depends on some other set of objects and therefore can be
+ * used against other equivalent versions of those objects. Essentially memoizes instance creation.
+ *
+ * @param <SeedObject> Object that determines the instance. The cache uses this object as a key for
+ *          its hash which is why it is imperative to have appropriate equals and hashcode
+ *          implementation for this object for the cache to work properly
+ * @param <Instance> Instance that will be created from SeedObject.
+ */
+public abstract class InstanceCache<SeedObject, Instance> {
+  private static final Logger LOG = LoggerFactory.getLogger(InstanceCache.class);
+  Map<SeedObject, Instance> cache = new HashMap<SeedObject, Instance>();
+
+  public InstanceCache() {}
+
+  /**
+   * Retrieve (or create if it doesn't exist) the correct Instance for this
+   * SeedObject
+   */
+  public Instance retrieve(SeedObject hv) throws Exception {
+    return retrieve(hv, null);
+  }
+
+  /**
+   * Retrieve (or create if it doesn't exist) the correct Instance for this
+   * SeedObject using 'seenSchemas' to resolve circular references
+   */
+  public synchronized Instance retrieve(SeedObject hv,
+      Set<SeedObject> seenSchemas) throws Exception {
+    if(LOG.isDebugEnabled()) LOG.debug("Checking for hv: " + hv.toString());
+
+    if(cache.containsKey(hv)) {
+      if(LOG.isDebugEnabled()) LOG.debug("Returning cache result.");
+      return cache.get(hv);
+    }
+
+    if(LOG.isDebugEnabled()) LOG.debug("Creating new instance and storing in cache");
+
+    Instance instance = makeInstance(hv, seenSchemas);
+    cache.put(hv, instance);
+    return instance;
+  }
+
+  protected abstract Instance makeInstance(SeedObject hv,
+      Set<SeedObject> seenSchemas) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaResolutionProblem.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaResolutionProblem.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaResolutionProblem.java
new file mode 100644
index 0000000..3470514
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaResolutionProblem.java
@@ -0,0 +1,62 @@
+/*
+ * 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.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.hive.metastore.utils.AvroSchemaUtils;
+
+@LimitedPrivate("Hive")
+public class SchemaResolutionProblem {
+  static final String sentinelString = "{\n" +
+        "    \"namespace\": \"org.apache.hadoop.hive\",\n" +
+        "    \"name\": \"CannotDetermineSchemaSentinel\",\n" +
+        "    \"type\": \"record\",\n" +
+        "    \"fields\": [\n" +
+        "        {\n" +
+        "            \"name\":\"ERROR_ERROR_ERROR_ERROR_ERROR_ERROR_ERROR\",\n" +
+        "            \"type\":\"string\"\n" +
+        "        },\n" +
+        "        {\n" +
+        "            \"name\":\"Cannot_determine_schema\",\n" +
+        "            \"type\":\"string\"\n" +
+        "        },\n" +
+        "        {\n" +
+        "            \"name\":\"check\",\n" +
+        "            \"type\":\"string\"\n" +
+        "        },\n" +
+        "        {\n" +
+        "            \"name\":\"schema\",\n" +
+        "            \"type\":\"string\"\n" +
+        "        },\n" +
+        "        {\n" +
+        "            \"name\":\"url\",\n" +
+        "            \"type\":\"string\"\n" +
+        "        },\n" +
+        "        {\n" +
+        "            \"name\":\"and\",\n" +
+        "            \"type\":\"string\"\n" +
+        "        },\n" +
+        "        {\n" +
+        "            \"name\":\"literal\",\n" +
+        "            \"type\":\"string\"\n" +
+        "        }\n" +
+        "    ]\n" +
+        "}";
+  public final static Schema SIGNAL_BAD_SCHEMA = AvroSchemaUtils.getSchemaFor(sentinelString);
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaToMetastoreTypeInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaToMetastoreTypeInfo.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaToMetastoreTypeInfo.java
new file mode 100644
index 0000000..b40a241
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/serde2/avro/SchemaToMetastoreTypeInfo.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.serde2.avro;
+
+import org.apache.hadoop.hive.serde2.typeinfo.MetastoreTypeInfoFactory;
+
+public class SchemaToMetastoreTypeInfo extends SchemaToTypeInfo {
+  private static final SchemaToMetastoreTypeInfo instance = new SchemaToMetastoreTypeInfo();
+  private SchemaToMetastoreTypeInfo() {
+    super(MetastoreTypeInfoFactory.getInstance());
+  }
+
+  public static final SchemaToMetastoreTypeInfo getInstance() {
+    return instance;
+  }
+}
\ No newline at end of file


[4/5] hive git commit: HIVE-17580 : Remove dependency of get_fields_with_environment_context API to serde (Vihang Karajgaonkar, reviewed by Alan Gates)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVarianceSample.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVarianceSample.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVarianceSample.java
index 75f28b6..c44e41b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVarianceSample.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVarianceSample.java
@@ -47,7 +47,7 @@ public class GenericUDAFVarianceSample extends GenericUDAFVariance {
           "Exactly one argument is expected.");
     }
 
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only primitive type arguments are accepted but "
           + parameters[0].getTypeName() + " is passed.");

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFnGrams.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFnGrams.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFnGrams.java
index cbf3f3f..c81b0cc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFnGrams.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFnGrams.java
@@ -67,7 +67,7 @@ public class GenericUDAFnGrams implements GenericUDAFResolver {
     // Validate the first parameter, which is the expression to compute over. This should be an
     // array of strings type, or an array of arrays of strings.
     PrimitiveTypeInfo pti;
-    if (parameters[0].getCategory() != ObjectInspector.Category.LIST) {
+    if (parameters[0].getCategory() != ObjectInspector.Category.LIST.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(0,
           "Only list type arguments are accepted but "
           + parameters[0].getTypeName() + " was passed as parameter 1.");
@@ -98,7 +98,7 @@ public class GenericUDAFnGrams implements GenericUDAFResolver {
     }
 
     // Validate the second parameter, which should be an integer
-    if(parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if(parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(1, "Only integers are accepted but "
           + parameters[1].getTypeName() + " was passed as parameter 2.");
     }
@@ -116,7 +116,7 @@ public class GenericUDAFnGrams implements GenericUDAFResolver {
     }
 
     // Validate the third parameter, which should also be an integer
-    if(parameters[2].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+    if(parameters[2].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       throw new UDFArgumentTypeException(2, "Only integers are accepted but "
             + parameters[2].getTypeName() + " was passed as parameter 3.");
     }
@@ -135,7 +135,7 @@ public class GenericUDAFnGrams implements GenericUDAFResolver {
 
     // If we have the optional fourth parameter, make sure it's also an integer
     if(parameters.length == 4) {
-      if(parameters[3].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+      if(parameters[3].getCategory() != ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
         throw new UDFArgumentTypeException(3, "Only integers are accepted but "
             + parameters[3].getTypeName() + " was passed as parameter 4.");
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFUtils.java
index c91865b..76ec346 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFUtils.java
@@ -253,7 +253,7 @@ public final class GenericUDFUtils {
     TypeInfo returnType = null;
     for (ExprNodeDesc node : children) {
       TypeInfo ti = node.getTypeInfo();
-      if (ti.getCategory() == Category.PRIMITIVE
+      if (ti.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()
         && ((PrimitiveTypeInfo)ti).getPrimitiveCategory() == PrimitiveCategory.VOID) {
         continue;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
index 83cdb2d..4523c87 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
@@ -65,7 +65,7 @@ public class TestVectorSerDeRow extends TestCase {
   private void verifyRead(
       DeserializeRead deserializeRead, TypeInfo typeInfo, Object expectedObject) throws IOException {
 
-    if (typeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE) {
+    if (typeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       VectorVerifyFast.verifyDeserializeRead(deserializeRead, typeInfo, expectedObject);
     } else {
       Object complexFieldObj = VectorVerifyFast.deserializeReadComplexType(deserializeRead, typeInfo);

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
index 3f99328..b546289 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
@@ -435,7 +435,7 @@ public class VectorRandomRowSource {
       }
 
       typeInfos[c] = typeInfo;
-      final Category category = typeInfo.getCategory();
+      final Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
       categories[c] = category;
       ObjectInspector objectInspector = getObjectInspector(typeInfo);
       switch (category) {
@@ -658,7 +658,7 @@ public class VectorRandomRowSource {
                 elementObjectInspector);
         boolean isStringFamily = false;
         PrimitiveCategory primitiveCategory = null;
-        if (elementTypeInfo.getCategory() == Category.PRIMITIVE) {
+        if (elementTypeInfo.getCategory() == Category.PRIMITIVE.toMetastoreTypeCategory()) {
           primitiveCategory = ((PrimitiveTypeInfo) elementTypeInfo).getPrimitiveCategory();
           if (primitiveCategory == PrimitiveCategory.STRING ||
               primitiveCategory == PrimitiveCategory.BINARY ||

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java
index 10ed6d7..11a3a52 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java
@@ -176,7 +176,7 @@ public class CheckFastRowHashMap extends CheckFastHashTable {
 
   private static void verifyRead(LazyBinaryDeserializeRead lazyBinaryDeserializeRead,
       TypeInfo typeInfo, Object expectedObject) throws IOException {
-    if (typeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE) {
+    if (typeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE.toMetastoreTypeCategory()) {
       VerifyFastRow.verifyDeserializeRead(lazyBinaryDeserializeRead, typeInfo, expectedObject);
     } else {
       final Object complexFieldObj =

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/ql/src/test/results/clientnegative/avro_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/avro_decimal.q.out b/ql/src/test/results/clientnegative/avro_decimal.q.out
index 9d00d6e..947be05 100644
--- a/ql/src/test/results/clientnegative/avro_decimal.q.out
+++ b/ql/src/test/results/clientnegative/avro_decimal.q.out
@@ -19,4 +19,4 @@ TBLPROPERTIES (
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@avro_dec
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.avro.AvroSerdeException Invalid precision or scale for decimal type)
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.avro.AvroSerdeException Decimal precision out of allowed range [1,38])

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/pom.xml
----------------------------------------------------------------------
diff --git a/serde/pom.xml b/serde/pom.xml
index 0247c32..b39c6b7 100644
--- a/serde/pom.xml
+++ b/serde/pom.xml
@@ -49,6 +49,11 @@
       <artifactId>hive-shims</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-standalone-metastore</artifactId>
+      <version>${project.version}</version>
+    </dependency>
     <!-- inter-project -->
     <dependency>
       <groupId>com.google.code.findbugs</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
index 085835f..1846fba 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
@@ -24,7 +24,6 @@ import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroObjectInspectorGenerator.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroObjectInspectorGenerator.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroObjectInspectorGenerator.java
index a07cf09..54a2fd0 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroObjectInspectorGenerator.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroObjectInspectorGenerator.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo;
 
 /**
@@ -50,7 +51,11 @@ public class AvroObjectInspectorGenerator {
     verifySchemaIsARecord(schema);
 
     this.columnNames = AvroObjectInspectorGenerator.generateColumnNames(schema);
-    this.columnTypes = SchemaToTypeInfo.generateColumnTypes(schema);
+    try {
+      this.columnTypes = SchemaToHiveTypeInfo.getInstance().generateColumnTypes(schema);
+    } catch (Exception e) {
+      throw new AvroSerdeException(e.getMessage());
+    }
     this.columnComments = AvroObjectInspectorGenerator.generateColumnComments(schema);
     assert columnNames.size() == columnTypes.size();
     this.oi = createObjectInspector();
@@ -139,7 +144,7 @@ public class AvroObjectInspectorGenerator {
   }
 
   private boolean supportedCategories(TypeInfo ti) {
-    final ObjectInspector.Category c = ti.getCategory();
+    final Category c = Category.fromMetastoreTypeCategory(ti.getCategory());
     return c.equals(ObjectInspector.Category.PRIMITIVE) ||
            c.equals(ObjectInspector.Category.MAP)       ||
            c.equals(ObjectInspector.Category.LIST)      ||

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerializer.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerializer.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerializer.java
index 83e5d68..e2043f6 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerializer.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerializer.java
@@ -165,7 +165,11 @@ class AvroSerializer {
         };
 
   private Object serializeEnum(TypeInfo typeInfo, PrimitiveObjectInspector fieldOI, Object structFieldData, Schema schema) throws AvroSerdeException {
-    return enums.retrieve(schema).retrieve(serializePrimitive(typeInfo, fieldOI, structFieldData, schema));
+    try {
+      return enums.retrieve(schema).retrieve(serializePrimitive(typeInfo, fieldOI, structFieldData, schema));
+    } catch (Exception e) {
+      throw new AvroSerdeException(e);
+    }
   }
 
   private Object serializeStruct(StructTypeInfo typeInfo, StructObjectInspector ssoi, Object o, Schema schema) throws AvroSerdeException {

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
deleted file mode 100644
index 2d52020..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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 java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Cache for objects whose creation only depends on some other set of objects and therefore can be
- * used against other equivalent versions of those objects. Essentially memoizes instance creation.
- *
- * @param <SeedObject> Object that determines the instance. The cache uses this object as a key for
- *          its hash which is why it is imperative to have appropriate equals and hashcode
- *          implementation for this object for the cache to work properly
- * @param <Instance> Instance that will be created from SeedObject.
- */
-public abstract class InstanceCache<SeedObject, Instance> {
-  private static final Logger LOG = LoggerFactory.getLogger(InstanceCache.class);
-  Map<SeedObject, Instance> cache = new HashMap<SeedObject, Instance>();
-  
-  public InstanceCache() {}
-
-  /**
-   * Retrieve (or create if it doesn't exist) the correct Instance for this
-   * SeedObject
-   */
-  public Instance retrieve(SeedObject hv) throws AvroSerdeException {
-    return retrieve(hv, null);
-  }
-
-  /**
-   * Retrieve (or create if it doesn't exist) the correct Instance for this
-   * SeedObject using 'seenSchemas' to resolve circular references
-   */
-  public synchronized Instance retrieve(SeedObject hv,
-      Set<SeedObject> seenSchemas) throws AvroSerdeException {
-    if(LOG.isDebugEnabled()) LOG.debug("Checking for hv: " + hv.toString());
-
-    if(cache.containsKey(hv)) {
-      if(LOG.isDebugEnabled()) LOG.debug("Returning cache result.");
-      return cache.get(hv);
-    }
-
-    if(LOG.isDebugEnabled()) LOG.debug("Creating new instance and storing in cache");
-
-    Instance instance = makeInstance(hv, seenSchemas);
-    cache.put(hv, instance);
-    return instance;
-  }
-
-  protected abstract Instance makeInstance(SeedObject hv,
-      Set<SeedObject> seenSchemas) throws AvroSerdeException;
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaResolutionProblem.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaResolutionProblem.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaResolutionProblem.java
deleted file mode 100644
index 65f104d..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaResolutionProblem.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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;
-
-class SchemaResolutionProblem {
-  static final String sentinelString = "{\n" +
-        "    \"namespace\": \"org.apache.hadoop.hive\",\n" +
-        "    \"name\": \"CannotDetermineSchemaSentinel\",\n" +
-        "    \"type\": \"record\",\n" +
-        "    \"fields\": [\n" +
-        "        {\n" +
-        "            \"name\":\"ERROR_ERROR_ERROR_ERROR_ERROR_ERROR_ERROR\",\n" +
-        "            \"type\":\"string\"\n" +
-        "        },\n" +
-        "        {\n" +
-        "            \"name\":\"Cannot_determine_schema\",\n" +
-        "            \"type\":\"string\"\n" +
-        "        },\n" +
-        "        {\n" +
-        "            \"name\":\"check\",\n" +
-        "            \"type\":\"string\"\n" +
-        "        },\n" +
-        "        {\n" +
-        "            \"name\":\"schema\",\n" +
-        "            \"type\":\"string\"\n" +
-        "        },\n" +
-        "        {\n" +
-        "            \"name\":\"url\",\n" +
-        "            \"type\":\"string\"\n" +
-        "        },\n" +
-        "        {\n" +
-        "            \"name\":\"and\",\n" +
-        "            \"type\":\"string\"\n" +
-        "        },\n" +
-        "        {\n" +
-        "            \"name\":\"literal\",\n" +
-        "            \"type\":\"string\"\n" +
-        "        }\n" +
-        "    ]\n" +
-        "}";
-  public final static Schema SIGNAL_BAD_SCHEMA = AvroSerdeUtils.getSchemaFor(sentinelString);
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaToHiveTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaToHiveTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaToHiveTypeInfo.java
new file mode 100644
index 0000000..d82f511
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaToHiveTypeInfo.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hive.serde2.avro;
+
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+public class SchemaToHiveTypeInfo extends SchemaToTypeInfo {
+  private static final SchemaToHiveTypeInfo instance = new SchemaToHiveTypeInfo();
+
+  private SchemaToHiveTypeInfo() {
+    //use getInstance to get this object. The base class uses cache to reuse
+    //Types when available
+    super(TypeInfoFactory.getInstance());
+  }
+
+  public static final SchemaToHiveTypeInfo getInstance() {
+    return instance;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaToTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaToTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaToTypeInfo.java
deleted file mode 100644
index 35d83bd..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/avro/SchemaToTypeInfo.java
+++ /dev/null
@@ -1,283 +0,0 @@
-/*
- * 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.hive.serde2.typeinfo.HiveDecimalUtils;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-
-/**
- * Convert an Avro Schema to a Hive TypeInfo
- */
-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 static final Map<Schema.Type, TypeInfo> primitiveTypeToTypeInfo = initTypeMap();
-  private static 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 AvroSerdeException for problems during conversion.
-   */
-  public static List<TypeInfo> generateColumnTypes(Schema schema) throws AvroSerdeException {
-    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 AvroSerdeException for problems during conversion.
-   */
-  public static List<TypeInfo> generateColumnTypes(Schema schema,
-      Set<Schema> seenSchemas) throws AvroSerdeException {
-    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;
-  }
-
-  static InstanceCache<Schema, TypeInfo> typeInfoCache = new InstanceCache<Schema, TypeInfo>() {
-                                  @Override
-                                  protected TypeInfo makeInstance(Schema s,
-                                      Set<Schema> seenSchemas)
-                                      throws AvroSerdeException {
-                                    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 AvroSerdeException for any problems during conversion.
-   */
-  public static TypeInfo generateTypeInfo(Schema schema,
-      Set<Schema> seenSchemas) throws AvroSerdeException {
-    // For bytes type, it can be mapped to decimal.
-    Schema.Type type = schema.getType();
-    if (type == BYTES && AvroSerDe.DECIMAL_TYPE_NAME
-      .equalsIgnoreCase(schema.getProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE))) {
-      int precision = 0;
-      int scale = 0;
-      try {
-        precision = schema.getJsonProp(AvroSerDe.AVRO_PROP_PRECISION).getIntValue();
-        scale = schema.getJsonProp(AvroSerDe.AVRO_PROP_SCALE).getIntValue();
-      } catch (Exception ex) {
-        throw new AvroSerdeException("Failed to obtain scale value from file schema: " + schema, ex);
-      }
-
-      try {
-        HiveDecimalUtils.validateParameter(precision, scale);
-      } catch (Exception ex) {
-        throw new AvroSerdeException("Invalid precision or scale for decimal type", ex);
-      }
-
-      return TypeInfoFactory.getDecimalTypeInfo(precision, scale);
-    }
-
-    if (type == STRING &&
-      AvroSerDe.CHAR_TYPE_NAME.equalsIgnoreCase(schema.getProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE))) {
-      int maxLength = 0;
-      try {
-        maxLength = schema.getJsonProp(AvroSerDe.AVRO_PROP_MAX_LENGTH).getValueAsInt();
-      } catch (Exception ex) {
-        throw new AvroSerdeException("Failed to obtain maxLength value from file schema: " + schema, ex);
-      }
-      return TypeInfoFactory.getCharTypeInfo(maxLength);
-    }
-
-    if (type == STRING && AvroSerDe.VARCHAR_TYPE_NAME
-      .equalsIgnoreCase(schema.getProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE))) {
-      int maxLength = 0;
-      try {
-        maxLength = schema.getJsonProp(AvroSerDe.AVRO_PROP_MAX_LENGTH).getValueAsInt();
-      } catch (Exception ex) {
-        throw new AvroSerdeException("Failed to obtain maxLength value from file schema: " + schema, ex);
-      }
-      return TypeInfoFactory.getVarcharTypeInfo(maxLength);
-    }
-
-    if (type == INT &&
-      AvroSerDe.DATE_TYPE_NAME.equals(schema.getProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE))) {
-      return TypeInfoFactory.dateTypeInfo;
-    }
-
-    if (type == LONG &&
-      AvroSerDe.TIMESTAMP_TYPE_NAME.equals(schema.getProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE))) {
-      return TypeInfoFactory.timestampTypeInfo;
-    }
-
-    return typeInfoCache.retrieve(schema, seenSchemas);
-  }
-
-  private static TypeInfo generateTypeInfoWorker(Schema schema,
-      Set<Schema> seenSchemas) throws AvroSerdeException {
-    // Avro requires NULLable types to be defined as unions of some type T
-    // and NULL.  This is annoying and we're going to hide it from the user.
-    if(AvroSerdeUtils.isNullableType(schema)) {
-      return generateTypeInfo(
-        AvroSerdeUtils.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 AvroSerdeException("Do not yet support: " + schema);
-    }
-  }
-
-  private static TypeInfo generateRecordTypeInfo(Schema schema,
-      Set<Schema> seenSchemas) throws AvroSerdeException {
-    assert schema.getType().equals(Schema.Type.RECORD);
-
-    if (seenSchemas == null) {
-        seenSchemas = Collections.newSetFromMap(new IdentityHashMap<Schema, Boolean>());
-    } else if (seenSchemas.contains(schema)) {
-      throw new AvroSerdeException(
-          "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 static TypeInfo generateMapTypeInfo(Schema schema,
-      Set<Schema> seenSchemas) throws AvroSerdeException {
-    assert schema.getType().equals(Schema.Type.MAP);
-    Schema valueType = schema.getValueType();
-    TypeInfo ti = generateTypeInfo(valueType, seenSchemas);
-
-    return TypeInfoFactory.getMapTypeInfo(TypeInfoFactory.getPrimitiveTypeInfo("string"), ti);
-  }
-
-  private static TypeInfo generateArrayTypeInfo(Schema schema,
-      Set<Schema> seenSchemas) throws AvroSerdeException {
-    assert schema.getType().equals(Schema.Type.ARRAY);
-    Schema itemsType = schema.getElementType();
-    TypeInfo itemsTypeInfo = generateTypeInfo(itemsType, seenSchemas);
-
-    return TypeInfoFactory.getListTypeInfo(itemsTypeInfo);
-  }
-
-  private static TypeInfo generateUnionTypeInfo(Schema schema,
-      Set<Schema> seenSchemas) throws AvroSerdeException {
-    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 static 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/serde/src/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java
deleted file mode 100644
index 4f8b05f..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java
+++ /dev/null
@@ -1,283 +0,0 @@
-/*
- * 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.serde2.objectinspector.PrimitiveObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-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.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
-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) {
-    PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) typeInfo;
-    Schema schema;
-    switch (primitiveTypeInfo.getPrimitiveCategory()) {
-      case STRING:
-        schema = Schema.create(Schema.Type.STRING);
-        break;
-      case CHAR:
-        schema = AvroSerdeUtils.getSchemaFor("{" +
-            "\"type\":\"" + AvroSerDe.AVRO_STRING_TYPE_NAME + "\"," +
-            "\"logicalType\":\"" + AvroSerDe.CHAR_TYPE_NAME + "\"," +
-            "\"maxLength\":" + ((CharTypeInfo) typeInfo).getLength() + "}");
-        break;
-      case VARCHAR:
-        schema = AvroSerdeUtils.getSchemaFor("{" +
-            "\"type\":\"" + AvroSerDe.AVRO_STRING_TYPE_NAME + "\"," +
-            "\"logicalType\":\"" + AvroSerDe.VARCHAR_TYPE_NAME + "\"," +
-            "\"maxLength\":" + ((VarcharTypeInfo) typeInfo).getLength() + "}");
-        break;
-      case BINARY:
-        schema = Schema.create(Schema.Type.BYTES);
-        break;
-      case BYTE:
-        schema = Schema.create(Schema.Type.INT);
-        break;
-      case SHORT:
-        schema = Schema.create(Schema.Type.INT);
-        break;
-      case INT:
-        schema = Schema.create(Schema.Type.INT);
-        break;
-      case LONG:
-        schema = Schema.create(Schema.Type.LONG);
-        break;
-      case FLOAT:
-        schema = Schema.create(Schema.Type.FLOAT);
-        break;
-      case DOUBLE:
-        schema = Schema.create(Schema.Type.DOUBLE);
-        break;
-      case BOOLEAN:
-        schema = Schema.create(Schema.Type.BOOLEAN);
-        break;
-      case DECIMAL:
-        DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo;
-        String precision = String.valueOf(decimalTypeInfo.precision());
-        String scale = String.valueOf(decimalTypeInfo.scale());
-        schema = AvroSerdeUtils.getSchemaFor("{" +
-            "\"type\":\"bytes\"," +
-            "\"logicalType\":\"decimal\"," +
-            "\"precision\":" + precision + "," +
-            "\"scale\":" + scale + "}");
-        break;
-      case DATE:
-        schema = AvroSerdeUtils.getSchemaFor("{" +
-            "\"type\":\"" + AvroSerDe.AVRO_INT_TYPE_NAME + "\"," +
-            "\"logicalType\":\"" + AvroSerDe.DATE_TYPE_NAME + "\"}");
-        break;
-      case TIMESTAMP:
-        schema = AvroSerdeUtils.getSchemaFor("{" +
-          "\"type\":\"" + AvroSerDe.AVRO_LONG_TYPE_NAME + "\"," +
-          "\"logicalType\":\"" + AvroSerDe.TIMESTAMP_TYPE_NAME + "\"}");
-        break;
-      case VOID:
-        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 (((PrimitiveTypeInfo) keyTypeInfo).getPrimitiveCategory()
-        != PrimitiveObjectInspector.PrimitiveCategory.STRING) {
-      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/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java
index 461043d..29751b0 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java
@@ -590,7 +590,7 @@ public final class BinarySortableDeserializeRead extends DeserializeRead {
 
   private Field createField(TypeInfo typeInfo) {
     final Field field = new Field();
-    final Category category = typeInfo.getCategory();
+    final Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
     field.category = category;
     field.typeInfo = typeInfo;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/fast/DeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/fast/DeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/fast/DeserializeRead.java
index 197031d..9adf29f 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/fast/DeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/fast/DeserializeRead.java
@@ -161,7 +161,7 @@ public abstract class DeserializeRead {
     primitiveCategories = new PrimitiveCategory[count];
     for (int i = 0; i < count; i++) {
       TypeInfo typeInfo = typeInfos[i];
-      Category category = typeInfo.getCategory();
+      Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
       categories[i] = category;
       if (category == Category.PRIMITIVE) {
         PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) typeInfo;

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
index 883ba88..e3ade95 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
@@ -325,7 +325,7 @@ public final class LazyFactory {
   public static ObjectInspector createLazyObjectInspector(TypeInfo typeInfo,
       int separatorIndex, LazyObjectInspectorParameters lazyParams,
       ObjectInspectorOptions option) throws SerDeException {
-    ObjectInspector.Category c = typeInfo.getCategory();
+    ObjectInspector.Category c = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
     switch (c) {
     case PRIMITIVE:
       return LazyPrimitiveObjectInspectorFactory.getLazyObjectInspector(

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
index fe0ee48..146954b 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
@@ -89,7 +89,7 @@ public final class LazySimpleDeserializeRead extends DeserializeRead {
     public ComplexTypeHelper complexTypeHelper;
 
     public Field(TypeInfo typeInfo, DataTypePhysicalVariation dataTypePhysicalVariation) {
-      Category category = typeInfo.getCategory();
+      Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
       if (category == Category.PRIMITIVE) {
         isPrimitive = true;
         primitiveCategory = ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory();

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
index 340f322..a23753f 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
@@ -112,7 +112,7 @@ public final class LazyBinaryDeserializeRead extends DeserializeRead {
 
   private Field createField(TypeInfo typeInfo) {
     final Field field = new Field();
-    final Category category = typeInfo.getCategory();
+    final Category category = Category.fromMetastoreTypeCategory(typeInfo.getCategory());
     field.category = category;
     field.typeInfo = typeInfo;
     switch (category) {

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspector.java
index 99b565d..2baa1fa 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspector.java
@@ -1,25 +1,27 @@
 /*
- * 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
+ *  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
  *
- * 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.
+ *       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;
 
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.serde2.typeinfo.MetastoreTypeCategory;
 
 /**
  * ObjectInspector helps us to look into the internal structure of a complex
@@ -47,7 +49,60 @@ public interface ObjectInspector extends Cloneable {
    *
    */
   public static enum Category {
-    PRIMITIVE, LIST, MAP, STRUCT, UNION
+    PRIMITIVE, LIST, MAP, STRUCT, UNION;
+
+    /**
+     * This utility method maps the MetastoreTypeCategory enum to Category enum.
+     * @param metastoreTypeCategory
+     * @return Category enum equivalent from MetastoreTypeCategory
+     */
+    public static Category fromMetastoreTypeCategory(MetastoreTypeCategory metastoreTypeCategory) {
+      switch (metastoreTypeCategory) {
+      case PRIMITIVE:
+        return Category.PRIMITIVE;
+      case LIST:
+        return Category.LIST;
+      case MAP:
+        return Category.MAP;
+      case STRUCT:
+        return Category.STRUCT;
+      case UNION:
+        return Category.UNION;
+      default:
+        throw new RuntimeException("Unsupported metastore type category " + metastoreTypeCategory);
+      }
+    }
+
+    /**
+     * returns the MetastoreTypeCategory enum mapping of this enum value
+     * @return
+     */
+    public MetastoreTypeCategory toMetastoreTypeCategory() {
+      switch (this) {
+      case PRIMITIVE:
+        return MetastoreTypeCategory.PRIMITIVE;
+      case LIST:
+        return MetastoreTypeCategory.LIST;
+      case MAP:
+        return MetastoreTypeCategory.MAP;
+      case STRUCT:
+        return MetastoreTypeCategory.STRUCT;
+      case UNION:
+        return MetastoreTypeCategory.UNION;
+      default:
+        throw new RuntimeException(
+            "Unsupported mapping to metastore type category " + this.toString());
+      }
+    }
+
+    /**
+     * Util method for mapping Category enum to MetastoreTypeCategory enum
+     * @param metastoreTypeCategory
+     * @return
+     */
+    public boolean equals(MetastoreTypeCategory metastoreTypeCategory) {
+      return this.toMetastoreTypeCategory().equals(metastoreTypeCategory);
+    }
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharTypeInfo.java
index 820fb4e..4ffe50f 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharTypeInfo.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharTypeInfo.java
@@ -67,4 +67,9 @@ public abstract class BaseCharTypeInfo extends PrimitiveTypeInfo {
     // type name should already be set by subclass
     return;
   }
+
+  @Override
+  public Object[] getParameters() {
+    return new Object[] { length };
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/DecimalTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/DecimalTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/DecimalTypeInfo.java
index 2e76df5..c5cb32b 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/DecimalTypeInfo.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/DecimalTypeInfo.java
@@ -127,4 +127,9 @@ public class DecimalTypeInfo extends PrimitiveTypeInfo {
     this.scale = scale;
   }
 
+  @Override
+  public Object[] getParameters() {
+    return new Object[] { precision, scale };
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java
deleted file mode 100644
index c632bff..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ListTypeInfo.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * 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.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-
-/**
- * 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 org.apache.hadoop.hive.serde.serdeConstants.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 Category getCategory() {
-    return Category.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/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java
deleted file mode 100644
index 1344c90..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/MapTypeInfo.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-
-/**
- * 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 org.apache.hadoop.hive.serde.serdeConstants.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 Category getCategory() {
-    return Category.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/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java
index 97af49a..d55e0c7 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 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;
@@ -36,16 +37,14 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class PrimitiveTypeInfo extends TypeInfo implements Serializable {
+public class PrimitiveTypeInfo extends MetastorePrimitiveTypeInfo implements Serializable {
   private static final long serialVersionUID = 1L;
 
-  // Base name (varchar vs fully qualified name such as varchar(200)).
-  protected String typeName;
-
   /**
    * For java serialization use only.
    */
   public PrimitiveTypeInfo() {
+    super();
   }
 
   /**
@@ -59,8 +58,8 @@ public class PrimitiveTypeInfo extends TypeInfo implements Serializable {
    * Returns the category of this TypeInfo.
    */
   @Override
-  public Category getCategory() {
-    return Category.PRIMITIVE;
+  public MetastoreTypeCategory getCategory() {
+    return MetastoreTypeCategory.PRIMITIVE;
   }
 
   public PrimitiveCategory getPrimitiveCategory() {
@@ -75,44 +74,7 @@ public class PrimitiveTypeInfo extends TypeInfo implements Serializable {
     return getPrimitiveTypeEntry().primitiveJavaClass;
   }
 
-  // The following 2 methods are for java serialization use only.
-  public void setTypeName(String typeName) {
-    this.typeName = typeName;
-  }
-
-  @Override
-  public String getTypeName() {
-    return typeName;
-  }
-
   public PrimitiveTypeEntry getPrimitiveTypeEntry() {
     return PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(typeName);
   }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other) {
-      return true;
-    }
-    if (other == null || getClass() != other.getClass()) {
-      return false;
-    }
-
-    PrimitiveTypeInfo pti = (PrimitiveTypeInfo) 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;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java
deleted file mode 100644
index 4caedb0..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/StructTypeInfo.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * 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.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-
-/**
- * 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(serdeConstants.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 Category getCategory() {
-    return Category.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/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TimestampLocalTZTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TimestampLocalTZTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TimestampLocalTZTypeInfo.java
index 6f9eeea..8b94208 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TimestampLocalTZTypeInfo.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TimestampLocalTZTypeInfo.java
@@ -101,4 +101,9 @@ public class TimestampLocalTZTypeInfo extends PrimitiveTypeInfo {
     this.timeZone = timeZone;
   }
 
+  @Override
+  public Object[] getParameters() {
+    return new Object[] { timeZone };
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/40ee74eb/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java
deleted file mode 100644
index 75e0973..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-
-/**
- * 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.Stable
-public abstract class TypeInfo implements Serializable {
-
-  private static final long serialVersionUID = 1L;
-
-  protected TypeInfo() {
-  }
-
-  /**
-   * The Category of this TypeInfo. Possible values are Primitive, List, Map,
-   * Struct and Union, which corresponds to the 5 sub-classes of TypeInfo.
-   */
-  public abstract Category getCategory();
-
-  /**
-   * A String representation of the TypeInfo.
-   */
-  public abstract String getTypeName();
-
-  /**
-   * 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/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
index 77d60c5..e730e9d 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoParser.PrimitiveParts;
 
 /**
  * TypeInfoFactory can be used to create the TypeInfo object for any types.
@@ -39,6 +40,53 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
  */
 public final class TypeInfoFactory {
 
+  //if this singleton looks weird, it is. It is done to support the move of TypeInfo and its sub-classes
+  //to standalone metastore. Standalone metastore needs access to TypeInfos without depending
+  //on hive-serde. In order to create these TypeInfo, it needs the TypeInfoParser and TypeInfoFactory.
+  //We cannot move TypeInfoFactory out of hive code base since its too disruptive.
+  //ITypeInfoFactory interface in standalone-metastore abstracts out the common functionality
+  //and metastore implements its own version of TypeInfoFactory. Hive uses its original TypeInfoFactory
+  //without any changes. The singleton weird-ness is needed to keep the caching behaviour when
+  //using this TypeInfoFactory in Deserializer and TypeInfoParsers
+  private static final ITypeInfoFactory instance = new ITypeInfoFactory() {
+    @Override
+    public MetastorePrimitiveTypeInfo getPrimitiveTypeInfo(String typeName, Object... parameters) {
+      if (serdeConstants.DECIMAL_TYPE_NAME.equals(typeName)) {
+        HiveDecimalUtils.validateParameter((Integer) parameters[0], (Integer) parameters[1]);
+      } else if (serdeConstants.CHAR_TYPE_NAME.equals(typeName)) {
+        BaseCharUtils.validateCharParameter((Integer) parameters[0]);
+      } else if (serdeConstants.VARCHAR_TYPE_NAME.equals(typeName)) {
+        BaseCharUtils.validateVarcharParameter((Integer)parameters[0]);
+      }
+      return TypeInfoFactory.getPrimitiveTypeInfo(
+          MetastoreTypeInfoUtils.getQualifiedPrimitiveTypeName(typeName, parameters));
+    }
+
+    @Override
+    public MapTypeInfo getMapTypeInfo(TypeInfo keyTypeInfo, TypeInfo valueTypeInfo) {
+      return (MapTypeInfo) TypeInfoFactory.getMapTypeInfo(keyTypeInfo, valueTypeInfo);
+    }
+
+    @Override
+    public ListTypeInfo getListTypeInfo(TypeInfo listElementTypeInfo) {
+      return (ListTypeInfo) TypeInfoFactory.getListTypeInfo(listElementTypeInfo);
+    }
+
+    @Override
+    public UnionTypeInfo getUnionTypeInfo(List<TypeInfo> typeInfos) {
+      return (UnionTypeInfo) TypeInfoFactory.getUnionTypeInfo(typeInfos);
+    }
+
+    @Override
+    public StructTypeInfo getStructTypeInfo(List<String> names, List<TypeInfo> typeInfos) {
+      return (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(names, typeInfos);
+    }
+  };
+
+  public static final ITypeInfoFactory getInstance() {
+    return instance;
+  }
+
   private TypeInfoFactory() {
     // prevent instantiation
   }
@@ -134,14 +182,14 @@ public final class TypeInfoFactory {
    * @return PrimitiveTypeInfo instance
    */
   private static PrimitiveTypeInfo createPrimitiveTypeInfo(String fullName) {
-    String baseName = TypeInfoUtils.getBaseName(fullName);
+    String baseName = MetastoreTypeInfoUtils.getBaseName(fullName);
     PrimitiveTypeEntry typeEntry =
         PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(baseName);
     if (null == typeEntry) {
       throw new RuntimeException("Unknown type " + fullName);
     }
 
-    TypeInfoUtils.PrimitiveParts parts = TypeInfoUtils.parsePrimitiveParts(fullName);
+    PrimitiveParts parts = TypeInfoUtils.parsePrimitiveParts(fullName);
     if (parts.typeParams == null || parts.typeParams.length < 1) {
       return null;
     }