http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
deleted file mode 100644
index 050dca9..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
+++ /dev/null
@@ -1,202 +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.metastore;
-
-import java.util.List;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-/**
- * Interface for Alter Table and Alter Partition code
- */
-public interface AlterHandler extends Configurable {
-
-  /**
-   * @deprecated As of release 2.2.0. Replaced by {@link #alterTable(RawStore, 
Warehouse, String,
-   * String, String, Table, EnvironmentContext, IHMSHandler)}
-   *
-   * handles alter table, the changes could be cascaded to partitions if 
applicable
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   *          Hive Warehouse where table data is stored
-   * @param catName
-   *          catalog of the table being altered
-   * @param dbname
-   *          database of the table being altered
-   * @param name
-   *          original name of the table being altered. same as
-   *          <i>newTable.tableName</i> if alter op is not a rename.
-   * @param newTable
-   *          new table object
-   * @throws InvalidOperationException
-   *           thrown if the newTable object is invalid
-   * @throws MetaException
-   *           thrown if there is any other error
-   */
-  @Deprecated
-  default void alterTable(RawStore msdb, Warehouse wh, String catName, String 
dbname,
-    String name, Table newTable, EnvironmentContext envContext)
-      throws InvalidOperationException, MetaException {
-    alterTable(msdb, wh, catName, dbname, name, newTable, envContext, null);
-  }
-
-  /**
-   * handles alter table, the changes could be cascaded to partitions if 
applicable
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   *          Hive Warehouse where table data is stored
-   * @param catName catalog of the table being altered
-   * @param dbname
-   *          database of the table being altered
-   * @param name
-   *          original name of the table being altered. same as
-   *          <i>newTable.tableName</i> if alter op is not a rename.
-   * @param newTable
-   *          new table object
-   * @param handler
-   *          HMSHandle object (required to log event notification)
-   * @throws InvalidOperationException
-   *           thrown if the newTable object is invalid
-   * @throws MetaException
-   *           thrown if there is any other error
-   */
-  void alterTable(RawStore msdb, Warehouse wh, String catName, String dbname,
-      String name, Table newTable, EnvironmentContext envContext,
-      IHMSHandler handler) throws InvalidOperationException, MetaException;
-
-  /**
-   * @deprecated As of release 2.2.0.  Replaced by {@link 
#alterPartition(RawStore, Warehouse, String,
-   * String, List, Partition, EnvironmentContext, IHMSHandler)}
-   *
-   * handles alter partition
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   * @param dbname
-   *          database of the partition being altered
-   * @param name
-   *          table of the partition being altered
-   * @param part_vals
-   *          original values of the partition being altered
-   * @param new_part
-   *          new partition object
-   * @return the altered partition
-   * @throws InvalidOperationException
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   */
-  @Deprecated
-  Partition alterPartition(final RawStore msdb, Warehouse wh, final String 
dbname,
-    final String name, final List<String> part_vals, final Partition new_part,
-    EnvironmentContext environmentContext)
-      throws InvalidOperationException, InvalidObjectException, 
AlreadyExistsException, MetaException;
-
-  /**
-   * handles alter partition
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh physical warehouse class
-   * @param catName catalog name
-   * @param dbname
-   *          database of the partition being altered
-   * @param name
-   *          table of the partition being altered
-   * @param part_vals
-   *          original values of the partition being altered
-   * @param new_part
-   *          new partition object
-   * @param handler
-   *          HMSHandle object (required to log event notification)
-   * @return the altered partition
-   * @throws InvalidOperationException
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   */
-  Partition alterPartition(final RawStore msdb, Warehouse wh, final String 
catName,
-                           final String dbname, final String name, final 
List<String> part_vals,
-                           final Partition new_part, EnvironmentContext 
environmentContext,
-                           IHMSHandler handler)
-      throws InvalidOperationException, InvalidObjectException, 
AlreadyExistsException, MetaException;
-
-  /**
-   * @deprecated As of release 3.0.0. Replaced by {@link 
#alterPartitions(RawStore, Warehouse, String,
-   * String, String, List, EnvironmentContext, IHMSHandler)}
-   *
-   * handles alter partitions
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   * @param dbname
-   *          database of the partition being altered
-   * @param name
-   *          table of the partition being altered
-   * @param new_parts
-   *          new partition list
-   * @return the altered partition list
-   * @throws InvalidOperationException
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   */
-  @Deprecated
-  List<Partition> alterPartitions(final RawStore msdb, Warehouse wh,
-    final String dbname, final String name, final List<Partition> new_parts,
-    EnvironmentContext environmentContext)
-      throws InvalidOperationException, InvalidObjectException, 
AlreadyExistsException, MetaException;
-
-  /**
-   * handles alter partitions
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   * @param dbname
-   *          database of the partition being altered
-   * @param name
-   *          table of the partition being altered
-   * @param new_parts
-   *          new partition list
-   * @param handler
-   *          HMSHandle object (required to log event notification)
-   * @return the altered partition list
-   * @throws InvalidOperationException
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   */
-  List<Partition> alterPartitions(final RawStore msdb, Warehouse wh, final 
String catName,
-    final String dbname, final String name, final List<Partition> new_parts,
-    EnvironmentContext environmentContext,IHMSHandler handler)
-      throws InvalidOperationException, InvalidObjectException, 
AlreadyExistsException, MetaException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Batchable.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Batchable.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Batchable.java
deleted file mode 100644
index 7e488a5..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Batchable.java
+++ /dev/null
@@ -1,86 +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.metastore;
-
-import java.util.ArrayList;
-import java.util.List;
-import javax.jdo.Query;
-
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *  Base class to add the batch process for DirectSQL or RawStore queries.
- *  1. Provide the implementation of run() to process one batch
- *  2. Call Batchable.runBatched() to process the whole dataset
- *
- *  I: input type, R: result type
- */
-public abstract class Batchable<I, R> {
-  private static final Logger LOG = LoggerFactory.getLogger(Batchable.class);
-  public static final int NO_BATCHING = -1;
-
-  private List<Query> queries = null;
-  public abstract List<R> run(List<I> input) throws MetaException;
-
-  public void addQueryAfterUse(Query query) {
-    if (queries == null) {
-      queries = new ArrayList<Query>(1);
-    }
-    queries.add(query);
-  }
-  protected void addQueryAfterUse(Batchable<?, ?> b) {
-    if (b.queries == null) {
-      return;
-    }
-    if (queries == null) {
-      queries = new ArrayList<Query>(1);
-    }
-    queries.addAll(b.queries);
-  }
-  public void closeAllQueries() {
-    for (Query q : queries) {
-      try {
-        q.closeAll();
-      } catch (Throwable t) {
-        LOG.error("Failed to close a query", t);
-      }
-    }
-  }
-
-  public static <I, R> List<R> runBatched(
-      final int batchSize,
-      List<I> input,
-      Batchable<I, R> runnable) throws MetaException {
-    if (batchSize == NO_BATCHING || batchSize >= input.size()) {
-      return runnable.run(input);
-    }
-    List<R> result = new ArrayList<R>(input.size());
-    for (int fromIndex = 0, toIndex = 0; toIndex < input.size(); fromIndex = 
toIndex) {
-      toIndex = Math.min(fromIndex + batchSize, input.size());
-      List<I> batchedInput = input.subList(fromIndex, toIndex);
-      List<R> batchedOutput = runnable.run(batchedInput);
-      if (batchedOutput != null) {
-        result.addAll(batchedOutput);
-      }
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
deleted file mode 100644
index d5dea4d..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
+++ /dev/null
@@ -1,301 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.utils.StringUtils;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Constants and utility functions for column types.  This is explicitly done 
as constants in the
- * class rather than an enum in order to interoperate with Hive's old 
serdeConstants.  All type
- * names in this class match the type names in Hive's serdeConstants class.  
They must continue
- * to do so.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class ColumnType {
-  public static final String VOID_TYPE_NAME = "void";
-
-  public static final String BOOLEAN_TYPE_NAME = "boolean";
-
-  public static final String TINYINT_TYPE_NAME = "tinyint";
-
-  public static final String SMALLINT_TYPE_NAME = "smallint";
-
-  public static final String INT_TYPE_NAME = "int";
-
-  public static final String BIGINT_TYPE_NAME = "bigint";
-
-  public static final String FLOAT_TYPE_NAME = "float";
-
-  public static final String DOUBLE_TYPE_NAME = "double";
-
-  public static final String STRING_TYPE_NAME = "string";
-
-  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 DATETIME_TYPE_NAME = "datetime";
-
-  public static final String TIMESTAMP_TYPE_NAME = "timestamp";
-
-  public static final String DECIMAL_TYPE_NAME = "decimal";
-
-  public static final String BINARY_TYPE_NAME = "binary";
-
-  public static final String INTERVAL_YEAR_MONTH_TYPE_NAME = 
"interval_year_month";
-
-  public static final String INTERVAL_DAY_TIME_TYPE_NAME = "interval_day_time";
-
-  public static final String TIMESTAMPTZ_TYPE_NAME = "timestamp with time 
zone";
-
-  public static final String LIST_TYPE_NAME = "array";
-
-  public static final String MAP_TYPE_NAME = "map";
-
-  public static final String STRUCT_TYPE_NAME = "struct";
-
-  public static final String UNION_TYPE_NAME = "uniontype";
-
-  public static final String LIST_COLUMNS = "columns";
-
-  public static final String LIST_COLUMN_TYPES = "columns.types";
-
-  public static final String COLUMN_NAME_DELIMITER = "column.name.delimiter";
-
-  public static final Set<String> PrimitiveTypes = StringUtils.asSet(
-    VOID_TYPE_NAME,
-    BOOLEAN_TYPE_NAME,
-    TINYINT_TYPE_NAME,
-    SMALLINT_TYPE_NAME,
-    INT_TYPE_NAME,
-    BIGINT_TYPE_NAME,
-    FLOAT_TYPE_NAME,
-    DOUBLE_TYPE_NAME,
-    STRING_TYPE_NAME,
-    VARCHAR_TYPE_NAME,
-    CHAR_TYPE_NAME,
-    DATE_TYPE_NAME,
-    DATETIME_TYPE_NAME,
-    TIMESTAMP_TYPE_NAME,
-    INTERVAL_YEAR_MONTH_TYPE_NAME,
-    INTERVAL_DAY_TIME_TYPE_NAME,
-    DECIMAL_TYPE_NAME,
-    BINARY_TYPE_NAME,
-    TIMESTAMPTZ_TYPE_NAME);
-
-  public static final Set<String> StringTypes = StringUtils.asSet(
-      STRING_TYPE_NAME,
-      VARCHAR_TYPE_NAME,
-      CHAR_TYPE_NAME
-  );
-
-  public static final Set<String> NumericTypes = StringUtils.asSet(
-      TINYINT_TYPE_NAME,
-      SMALLINT_TYPE_NAME,
-      INT_TYPE_NAME,
-      BIGINT_TYPE_NAME,
-      FLOAT_TYPE_NAME,
-      DOUBLE_TYPE_NAME,
-      DECIMAL_TYPE_NAME
-  );
-
-  // This intentionally does not include interval types.
-  public static final Set<String> DateTimeTypes = StringUtils.asSet(
-      DATE_TYPE_NAME,
-      DATETIME_TYPE_NAME,
-      TIMESTAMP_TYPE_NAME,
-      TIMESTAMPTZ_TYPE_NAME
-  );
-
-  // This map defines the progression of up casts in numeric types.
-  public static final Map<String, Integer> NumericCastOrder = new HashMap<>();
-
-  static {
-    NumericCastOrder.put(TINYINT_TYPE_NAME, 1);
-    NumericCastOrder.put(SMALLINT_TYPE_NAME, 2);
-    NumericCastOrder.put(INT_TYPE_NAME, 3);
-    NumericCastOrder.put(BIGINT_TYPE_NAME, 4);
-    NumericCastOrder.put(DECIMAL_TYPE_NAME, 5);
-    NumericCastOrder.put(FLOAT_TYPE_NAME, 6);
-    NumericCastOrder.put(DOUBLE_TYPE_NAME, 7);
-  }
-
-  private static final Map<String, String> alternateTypeNames = new 
HashMap<>();
-
-  static {
-    alternateTypeNames.put("integer", INT_TYPE_NAME);
-    alternateTypeNames.put("numeric", DECIMAL_TYPE_NAME);
-  }
-
-  public static final Set<String> CollectionTypes = StringUtils.asSet(
-    LIST_TYPE_NAME,
-    MAP_TYPE_NAME);
-
-  public static final Set<String> IntegralTypes = StringUtils.asSet(
-    TINYINT_TYPE_NAME,
-    SMALLINT_TYPE_NAME,
-    INT_TYPE_NAME,
-    BIGINT_TYPE_NAME);
-
-  public static final Set<String> AllTypes = StringUtils.asSet(
-    VOID_TYPE_NAME,
-    BOOLEAN_TYPE_NAME,
-    TINYINT_TYPE_NAME,
-    SMALLINT_TYPE_NAME,
-    INT_TYPE_NAME,
-    BIGINT_TYPE_NAME,
-    FLOAT_TYPE_NAME,
-    DOUBLE_TYPE_NAME,
-    STRING_TYPE_NAME,
-    CHAR_TYPE_NAME,
-    VARCHAR_TYPE_NAME,
-    DATE_TYPE_NAME,
-    DATETIME_TYPE_NAME,
-    TIMESTAMP_TYPE_NAME,
-    DECIMAL_TYPE_NAME,
-    BINARY_TYPE_NAME,
-    INTERVAL_YEAR_MONTH_TYPE_NAME,
-    INTERVAL_DAY_TIME_TYPE_NAME,
-    TIMESTAMPTZ_TYPE_NAME,
-    LIST_TYPE_NAME,
-    MAP_TYPE_NAME,
-    STRUCT_TYPE_NAME,
-    UNION_TYPE_NAME,
-    LIST_COLUMNS,
-    LIST_COLUMN_TYPES,
-    COLUMN_NAME_DELIMITER
-  );
-
-  /**
-   * Given a type string return the type name.  For example, passing in the 
type string
-   * <tt>varchar(256)</tt> will return <tt>varchar</tt>.
-   * @param typeString Type string
-   * @return type name, guaranteed to be in lower case
-   */
-  public static String getTypeName(String typeString) {
-    if (typeString == null) return null;
-    String protoType = typeString.toLowerCase().split("\\W")[0];
-    String realType = alternateTypeNames.get(protoType);
-    return realType == null ? protoType : realType;
-  }
-
-  public static boolean areColTypesCompatible(String from, String to) {
-    if (from.equals(to)) return true;
-
-    if (PrimitiveTypes.contains(from) && PrimitiveTypes.contains(to)) {
-      // They aren't the same, but we may be able to do a cast
-
-      // If they are both types of strings, that should be fine
-      if (StringTypes.contains(from) && StringTypes.contains(to)) return true;
-
-      // If both are numeric, make sure the new type is larger than the old.
-      if (NumericTypes.contains(from) && NumericTypes.contains(to)) {
-        return NumericCastOrder.get(from) < NumericCastOrder.get(to);
-      }
-
-      // Allow string to double conversion
-      if (StringTypes.contains(from) && to.equals(DOUBLE_TYPE_NAME)) return 
true;
-
-      // Void can go to anything
-      if (from.equals(VOID_TYPE_NAME)) return true;
-
-      // Allow date to string casts.  NOTE: I suspect this is the reverse of 
what we actually
-      // want, but it matches the code in 
o.a.h.h.serde2.typeinfo.TypeInfoUtils.  I can't see how
-      // users would be altering date columns into string columns.  The other 
I easily see since
-      // Hive did not originally support datetime types.  Also, the comment in 
the Hive code
-      // says string to date, even though the code does the opposite.  But for 
now I'm keeping
-      // this as is so the functionality matches.
-      if (DateTimeTypes.contains(from) && StringTypes.contains(to)) return 
true;
-
-      // Allow numeric to string
-      if (NumericTypes.contains(from) && StringTypes.contains(to)) return true;
-
-    }
-    return false;
-  }
-
-  // These aren't column types, they are info for how things are stored in 
thrift.
-  // It didn't seem useful to create another Constants class just for these 
though.
-  public static final String SERIALIZATION_FORMAT = "serialization.format";
-
-  public static final String SERIALIZATION_LIB = "serialization.lib";
-
-  public static final String SERIALIZATION_DDL = "serialization.ddl";
-
-  public static final char COLUMN_COMMENTS_DELIMITER = '\0';
-
-  private static HashMap<String, String> typeToThriftTypeMap;
-  static {
-    typeToThriftTypeMap = new HashMap<>();
-    typeToThriftTypeMap.put(BOOLEAN_TYPE_NAME, "bool");
-    typeToThriftTypeMap.put(TINYINT_TYPE_NAME, "byte");
-    typeToThriftTypeMap.put(SMALLINT_TYPE_NAME, "i16");
-    typeToThriftTypeMap.put(INT_TYPE_NAME, "i32");
-    typeToThriftTypeMap.put(BIGINT_TYPE_NAME, "i64");
-    typeToThriftTypeMap.put(DOUBLE_TYPE_NAME, "double");
-    typeToThriftTypeMap.put(FLOAT_TYPE_NAME, "float");
-    typeToThriftTypeMap.put(LIST_TYPE_NAME, "list");
-    typeToThriftTypeMap.put(MAP_TYPE_NAME, "map");
-    typeToThriftTypeMap.put(STRING_TYPE_NAME, "string");
-    typeToThriftTypeMap.put(BINARY_TYPE_NAME, "binary");
-    // These 4 types are not supported yet.
-    // We should define a complex type date in thrift that contains a single 
int
-    // member, and DynamicSerDe
-    // should convert it to date type at runtime.
-    typeToThriftTypeMap.put(DATE_TYPE_NAME, "date");
-    typeToThriftTypeMap.put(DATETIME_TYPE_NAME, "datetime");
-    typeToThriftTypeMap.put(TIMESTAMP_TYPE_NAME, "timestamp");
-    typeToThriftTypeMap.put(DECIMAL_TYPE_NAME, "decimal");
-    typeToThriftTypeMap.put(INTERVAL_YEAR_MONTH_TYPE_NAME, 
INTERVAL_YEAR_MONTH_TYPE_NAME);
-    typeToThriftTypeMap.put(INTERVAL_DAY_TIME_TYPE_NAME, 
INTERVAL_DAY_TIME_TYPE_NAME);
-  }
-
-  /**
-   * Convert type to ThriftType. We do that by tokenizing the type and convert
-   * each token.
-   */
-  public static String typeToThriftType(String type) {
-    StringBuilder thriftType = new StringBuilder();
-    int last = 0;
-    boolean lastAlphaDigit = Character.isLetterOrDigit(type.charAt(last));
-    for (int i = 1; i <= type.length(); i++) {
-      if (i == type.length()
-          || Character.isLetterOrDigit(type.charAt(i)) != lastAlphaDigit) {
-        String token = type.substring(last, i);
-        last = i;
-        String thriftToken = typeToThriftTypeMap.get(token);
-        thriftType.append(thriftToken == null ? token : thriftToken);
-        lastAlphaDigit = !lastAlphaDigit;
-      }
-    }
-    return thriftType.toString();
-  }
-
-  public static String getListType(String t) {
-    return "array<" + t + ">";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
deleted file mode 100644
index 0b3504d..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
+++ /dev/null
@@ -1,75 +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.metastore;
-
-import java.sql.SQLException;
-import java.sql.SQLTransactionRollbackException;
-
-/** Database product infered via JDBC. */
-public enum DatabaseProduct {
-  DERBY, MYSQL, POSTGRES, ORACLE, SQLSERVER, OTHER;
-
-  /**
-   * Determine the database product type
-   * @param productName string to defer database connection
-   * @return database product type
-   */
-  public static DatabaseProduct determineDatabaseProduct(String productName) 
throws SQLException {
-    if (productName == null) {
-      return OTHER;
-    }
-    productName = productName.toLowerCase();
-    if (productName.contains("derby")) {
-      return DERBY;
-    } else if (productName.contains("microsoft sql server")) {
-      return SQLSERVER;
-    } else if (productName.contains("mysql")) {
-      return MYSQL;
-    } else if (productName.contains("oracle")) {
-      return ORACLE;
-    } else if (productName.contains("postgresql")) {
-      return POSTGRES;
-    } else {
-      return OTHER;
-    }
-  }
-
-  public static boolean isDeadlock(DatabaseProduct dbProduct, SQLException e) {
-    return e instanceof SQLTransactionRollbackException
-        || ((dbProduct == MYSQL || dbProduct == POSTGRES || dbProduct == 
SQLSERVER)
-            && "40001".equals(e.getSQLState()))
-        || (dbProduct == POSTGRES && "40P01".equals(e.getSQLState()))
-        || (dbProduct == ORACLE && (e.getMessage() != null && 
(e.getMessage().contains("deadlock detected")
-            || e.getMessage().contains("can't serialize access for this 
transaction"))));
-  }
-
-  /**
-   * Whether the RDBMS has restrictions on IN list size (explicit, or poor 
perf-based).
-   */
-  public static boolean needsInBatching(DatabaseProduct dbType) {
-    return dbType == ORACLE || dbType == SQLSERVER;
-  }
-
-  /**
-   * Whether the RDBMS has a bug in join and filter operation order described 
in DERBY-6358.
-   */
-  public static boolean hasJoinOperationOrderBug(DatabaseProduct dbType) {
-    return dbType == DERBY || dbType == ORACLE;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Deadline.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Deadline.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Deadline.java
deleted file mode 100644
index 2e00005..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Deadline.java
+++ /dev/null
@@ -1,172 +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.metastore;
-
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-
-/**
- * This class is used to monitor long running methods in a thread.
- * It is recommended to use it as a ThreadLocal variable.
- */
-public class Deadline {
-  private static final Logger LOG = 
LoggerFactory.getLogger(Deadline.class.getName());
-
-  /**
-   * its value is init from conf, and could be reset from client.
-   */
-  private long timeoutNanos;
-
-  /**
-   * it is reset before executing a method
-   */
-  private long startTime = NO_DEADLINE;
-
-  /**
-   * The name of public methods in HMSHandler
-   */
-  private String method;
-
-  private Deadline(long timeoutMs) {
-    this.timeoutNanos = timeoutMs * 1000000L;
-  }
-
-  /**
-   * Deadline object per thread.
-   */
-  private static final ThreadLocal<Deadline> DEADLINE_THREAD_LOCAL = new 
ThreadLocal<Deadline>() {
-        @Override
-        protected Deadline initialValue() {
-          return null;
-        }
-      };
-
-  private static void setCurrentDeadline(Deadline deadline) {
-    DEADLINE_THREAD_LOCAL.set(deadline);
-  }
-
-  static Deadline getCurrentDeadline() {
-    return DEADLINE_THREAD_LOCAL.get();
-  }
-
-  private static void removeCurrentDeadline() {
-    DEADLINE_THREAD_LOCAL.remove();
-  }
-
-  /**
-   * register a Deadline threadlocal object to current thread.
-   * @param timeout
-   */
-  public static void registerIfNot(long timeout) {
-    if (getCurrentDeadline() == null) {
-      setCurrentDeadline(new Deadline(timeout));
-    }
-  }
-
-  /**
-   * reset the timeout value of this timer.
-   * @param timeoutMs
-   */
-  public static void resetTimeout(long timeoutMs) throws MetaException {
-    if (timeoutMs <= 0) {
-      throw MetaStoreUtils.newMetaException(new DeadlineException("The reset 
timeout value should be " +
-          "larger than 0: " + timeoutMs));
-    }
-    Deadline deadline = getCurrentDeadline();
-    if (deadline != null) {
-      deadline.timeoutNanos = timeoutMs * 1000000L;
-    } else {
-      throw MetaStoreUtils.newMetaException(new DeadlineException("The 
threadlocal Deadline is null," +
-          " please register it first."));
-    }
-  }
-
-  /**
-   * start the timer before a method is invoked.
-   * @param method method to be invoked
-   */
-  public static boolean startTimer(String method) throws MetaException {
-    Deadline deadline = getCurrentDeadline();
-    if (deadline == null) {
-      throw MetaStoreUtils.newMetaException(new DeadlineException("The 
threadlocal Deadline is null," +
-          " please register it first."));
-    }
-    if (deadline.startTime != NO_DEADLINE) return false;
-    deadline.method = method;
-    do {
-      deadline.startTime = System.nanoTime();
-    } while (deadline.startTime == NO_DEADLINE);
-    return true;
-  }
-
-  /**
-   * end the time after a method is done.
-   */
-  public static void stopTimer() throws MetaException {
-    Deadline deadline = getCurrentDeadline();
-    if (deadline != null) {
-      deadline.startTime = NO_DEADLINE;
-      deadline.method = null;
-    } else {
-      throw MetaStoreUtils.newMetaException(new DeadlineException("The 
threadlocal Deadline is null," +
-          " please register it first."));
-    }
-  }
-
-  /**
-   * remove the registered Deadline threadlocal object from current thread.
-   */
-  public static void clear() {
-    removeCurrentDeadline();
-  }
-
-  /**
-   * Check whether the long running method timeout.
-   * @throws MetaException when the method timeout
-   */
-  public static void checkTimeout() throws MetaException {
-    Deadline deadline = getCurrentDeadline();
-    if (deadline != null) {
-      deadline.check();
-    } else {
-      throw MetaStoreUtils.newMetaException(new DeadlineException("The 
threadlocal Deadline is null," +
-          " please register it first."));
-    }
-  }
-
-  private static final long NO_DEADLINE = Long.MIN_VALUE;
-
-  private void check() throws MetaException{
-    try {
-      if (startTime == NO_DEADLINE) {
-        throw new DeadlineException("Should execute startTimer() method before 
" +
-            "checkTimeout. Error happens in method: " + method);
-      }
-      long elapsedTime = System.nanoTime() - startTime;
-      if (elapsedTime > timeoutNanos) {
-        throw new DeadlineException("Timeout when executing method: " + method 
+ "; "
-            + (elapsedTime / 1000000L) + "ms exceeds " + (timeoutNanos / 
1000000L)  + "ms");
-      }
-    } catch (DeadlineException e) {
-      throw MetaStoreUtils.newMetaException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DeadlineException.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DeadlineException.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DeadlineException.java
deleted file mode 100644
index b18c396..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DeadlineException.java
+++ /dev/null
@@ -1,29 +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.metastore;
-
-/**
- * Thrown when a long running method timeout is checked.
- */
-public class DeadlineException extends Exception {
-
-  public DeadlineException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DefaultPartitionExpressionProxy.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DefaultPartitionExpressionProxy.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DefaultPartitionExpressionProxy.java
deleted file mode 100644
index ec543be..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DefaultPartitionExpressionProxy.java
+++ /dev/null
@@ -1,57 +0,0 @@
-package org.apache.hadoop.hive.metastore;
-/*
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.
- */
-
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-
-import java.util.List;
-
-/**
- * Default implementation of PartitionExpressionProxy.  Eventually this should 
use the SARGs in
- * Hive's storage-api.  For now it just throws UnsupportedOperationException.
- */
-public class DefaultPartitionExpressionProxy implements 
PartitionExpressionProxy {
-  @Override
-  public String convertExprToFilter(byte[] expr) throws MetaException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean filterPartitionsByExpr(List<FieldSchema> partColumns, byte[] 
expr, String
-      defaultPartitionName, List<String> partitionNames) throws MetaException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public FileMetadataExprType getMetadataType(String inputFormat) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public FileFormatProxy getFileFormatProxy(FileMetadataExprType type) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public SearchArgument createSarg(byte[] expr) {
-    throw new UnsupportedOperationException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DefaultStorageSchemaReader.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DefaultStorageSchemaReader.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DefaultStorageSchemaReader.java
deleted file mode 100644
index 1dbfa42..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/DefaultStorageSchemaReader.java
+++ /dev/null
@@ -1,38 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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 java.util.List;
-
-/**
- * Default StorageSchemaReader.  This just throws as the metastore currently 
doesn't know how to
- * read schemas from storage.
- */
-public class DefaultStorageSchemaReader implements StorageSchemaReader {
-  @Override
-  public List<FieldSchema> readSchema(Table tbl, EnvironmentContext envContext,
-                                      Configuration conf) throws MetaException 
{
-    throw new UnsupportedOperationException("Storage schema reading not 
supported");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataHandler.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataHandler.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataHandler.java
deleted file mode 100644
index ff30260..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataHandler.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.metastore;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The base implementation of a file metadata handler for a specific file type.
- * There are currently two classes for each file type (of 1), this one, which 
is very simple due
- * to the fact that it just calls the proxy class for most calls; and the 
proxy class, that
- * contains the actual implementation that depends on some stuff in QL (for 
ORC).
- */
-public abstract class FileMetadataHandler {
-  protected static final Logger LOG = 
LoggerFactory.getLogger(FileMetadataHandler.class);
-
-  private Configuration conf;
-  private PartitionExpressionProxy expressionProxy;
-  private FileFormatProxy fileFormatProxy;
-  private MetadataStore store;
-
-  /**
-   * Same as RawStore.getFileMetadataByExpr.
-   */
-  public abstract void getFileMetadataByExpr(List<Long> fileIds, byte[] expr,
-      ByteBuffer[] metadatas, ByteBuffer[] results, boolean[] eliminated) 
throws IOException;
-
-  protected abstract FileMetadataExprType getType();
-
-  protected PartitionExpressionProxy getExpressionProxy() {
-    return expressionProxy;
-  }
-
-  protected FileFormatProxy getFileFormatProxy() {
-    return fileFormatProxy;
-  }
-
-  protected MetadataStore getStore() {
-    return store;
-  }
-
-  /**
-   * Configures the handler. Called once before use.
-   * @param conf Config.
-   * @param expressionProxy Expression proxy to access ql stuff.
-   * @param store Storage interface to manipulate the metadata.
-   */
-  public void configure(
-      Configuration conf, PartitionExpressionProxy expressionProxy, 
MetadataStore store) {
-    this.conf = conf;
-    this.expressionProxy = expressionProxy;
-    this.store = store;
-    this.fileFormatProxy = expressionProxy.getFileFormatProxy(getType());
-  }
-
-  /**
-   * Caches the file metadata for a particular file.
-   * @param fileId File id.
-   * @param fs The filesystem of the file.
-   * @param path Path to the file.
-   */
-  public void cacheFileMetadata(long fileId, FileSystem fs, Path path)
-      throws IOException, InterruptedException {
-    // ORC is in ql, so we cannot do anything here. For now, all the logic is 
in the proxy.
-    ByteBuffer[] cols = fileFormatProxy.getAddedColumnsToCache();
-    ByteBuffer[] vals = (cols == null) ? null : new ByteBuffer[cols.length];
-    ByteBuffer metadata = fileFormatProxy.getMetadataToCache(fs, path, vals);
-    LOG.info("Caching file metadata for " + path + ", size " + 
metadata.remaining());
-    store.storeFileMetadata(fileId, metadata, cols, vals);
-  }
-
-  /**
-   * @return the added column names to be cached in metastore with the 
metadata for this type.
-   */
-  public ByteBuffer[] createAddedCols() {
-    return fileFormatProxy.getAddedColumnsToCache();
-  }
-
-  /**
-   * @return the values for the added columns returned by createAddedCols for 
respective metadatas.
-   */
-  public ByteBuffer[][] createAddedColVals(List<ByteBuffer> metadata) {
-    return fileFormatProxy.getAddedValuesToCache(metadata);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
deleted file mode 100644
index 8204829..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
+++ /dev/null
@@ -1,119 +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.metastore;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.utils.HdfsUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-public class FileMetadataManager {
-  private static final Logger LOG = 
LoggerFactory.getLogger(FileMetadataManager.class);
-
-  private final RawStore tlms;
-  private final ExecutorService threadPool;
-  private final Configuration conf;
-
-  private final class CacheUpdateRequest implements Callable<Void> {
-    FileMetadataExprType type;
-    String location;
-
-    public CacheUpdateRequest(FileMetadataExprType type, String location) {
-      this.type = type;
-      this.location = location;
-    }
-
-    @Override
-    public Void call() throws Exception {
-      try {
-        cacheMetadata(type, location);
-      } catch (InterruptedException ex) {
-        Thread.currentThread().interrupt();
-      } catch (Exception ex) {
-        // Nobody can see this exception on the threadpool; just log it.
-        LOG.error("Failed to cache file metadata in background for " + type + 
", " + location, ex);
-      }
-      return null;
-    }
-  }
-
-  public FileMetadataManager(RawStore tlms, Configuration conf) {
-    this.tlms = tlms;
-    this.conf = conf;
-    int numThreads = MetastoreConf.getIntVar(conf, 
MetastoreConf.ConfVars.FILE_METADATA_THREADS);
-    this.threadPool = Executors.newFixedThreadPool(numThreads,
-        new 
ThreadFactoryBuilder().setNameFormat("File-Metadata-%d").setDaemon(true).build());
-  }
-
-  public void queueCacheMetadata(String location, FileMetadataExprType type) {
-    threadPool.submit(new CacheUpdateRequest(type, location));
-  }
-
-  private void cacheMetadata(FileMetadataExprType type, String location)
-      throws MetaException, IOException, InterruptedException {
-    Path path = new Path(location);
-    FileSystem fs = path.getFileSystem(conf);
-    List<Path> files;
-    if (!fs.isDirectory(path)) {
-      files = Lists.newArrayList(path);
-    } else {
-      files = new ArrayList<>();
-      RemoteIterator<LocatedFileStatus> iter = fs.listFiles(path, true);
-      while (iter.hasNext()) {
-        // TODO: use fileId right from the list after HDFS-7878; or get dfs 
client and do it
-        LocatedFileStatus lfs = iter.next();
-        if (lfs.isDirectory()) continue;
-        files.add(lfs.getPath());
-      }
-    }
-    for (Path file : files) {
-      long fileId;
-      // TODO: use the other HdfsUtils here
-      if (!(fs instanceof DistributedFileSystem)) return;
-      try {
-        fileId = HdfsUtils.getFileId(fs, 
Path.getPathWithoutSchemeAndAuthority(file).toString());
-      } catch (UnsupportedOperationException ex) {
-        LOG.error("Cannot cache file metadata for " + location + "; "
-            + fs.getClass().getCanonicalName() + " does not support fileId");
-        return;
-      }
-      LOG.info("Caching file metadata for " + file + " (file ID " + fileId + 
")");
-      file = HdfsUtils.getFileIdPath(fs, file, fileId);
-      tlms.getFileMetadataHandler(type).cacheFileMetadata(fileId, fs, file);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/081fa368/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
deleted file mode 100644
index 1ee6d97..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
+++ /dev/null
@@ -1,90 +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.metastore;
-
-import com.codahale.metrics.Counter;
-import com.codahale.metrics.MetricRegistry;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
-import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.DropTableEvent;
-import org.apache.hadoop.hive.metastore.metrics.Metrics;
-import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Report metrics of metadata added, deleted by this Hive Metastore.
- */
-public class HMSMetricsListener extends MetaStoreEventListener {
-
-  private static final Logger LOGGER = 
LoggerFactory.getLogger(HMSMetricsListener.class);
-
-  private Counter createdDatabases, deletedDatabases, createdTables, 
deletedTables, createdParts,
-      deletedParts;
-
-  public HMSMetricsListener(Configuration config) {
-    super(config);
-    createdDatabases = 
Metrics.getOrCreateCounter(MetricsConstants.CREATE_TOTAL_DATABASES);
-    deletedDatabases = 
Metrics.getOrCreateCounter(MetricsConstants.DELETE_TOTAL_DATABASES);
-    createdTables = 
Metrics.getOrCreateCounter(MetricsConstants.CREATE_TOTAL_TABLES);
-    deletedTables = 
Metrics.getOrCreateCounter(MetricsConstants.DELETE_TOTAL_TABLES);
-    createdParts = 
Metrics.getOrCreateCounter(MetricsConstants.CREATE_TOTAL_PARTITIONS);
-    deletedParts = 
Metrics.getOrCreateCounter(MetricsConstants.DELETE_TOTAL_PARTITIONS);
-  }
-
-  @Override
-  public void onCreateDatabase(CreateDatabaseEvent dbEvent) throws 
MetaException {
-    
Metrics.getOrCreateGauge(MetricsConstants.TOTAL_DATABASES).incrementAndGet();
-    createdDatabases.inc();
-  }
-
-  @Override
-  public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException {
-    
Metrics.getOrCreateGauge(MetricsConstants.TOTAL_DATABASES).decrementAndGet();
-    deletedDatabases.inc();
-  }
-
-  @Override
-  public void onCreateTable(CreateTableEvent tableEvent) throws MetaException {
-    Metrics.getOrCreateGauge(MetricsConstants.TOTAL_TABLES).incrementAndGet();
-    createdTables.inc();
-  }
-
-  @Override
-  public void onDropTable(DropTableEvent tableEvent) throws MetaException {
-    Metrics.getOrCreateGauge(MetricsConstants.TOTAL_TABLES).decrementAndGet();
-    deletedTables.inc();
-  }
-
-  @Override
-  public void onDropPartition(DropPartitionEvent partitionEvent) throws 
MetaException {
-    
Metrics.getOrCreateGauge(MetricsConstants.TOTAL_PARTITIONS).decrementAndGet();
-    deletedParts.inc();
-  }
-
-  @Override
-  public void onAddPartition(AddPartitionEvent partitionEvent) throws 
MetaException {
-    
Metrics.getOrCreateGauge(MetricsConstants.TOTAL_PARTITIONS).incrementAndGet();
-    createdParts.inc();
-  }
-}

Reply via email to