소스 검색

HADOOP-4205. hive: metastore and ql to use the refactored SerDe library.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@697291 13f79535-47bb-0310-9956-ffa450edef68
Zheng Shao 16 년 전
부모
커밋
664fd2ea1c
100개의 변경된 파일3312개의 추가작업 그리고 8318개의 파일을 삭제
  1. 3 0
      CHANGES.txt
  2. 23 22
      src/contrib/hive/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
  3. 138 185
      src/contrib/hive/metastore/if/hive_metastore.thrift
  4. 0 53
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Constants.java
  5. 28 28
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
  6. 6 6
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java
  7. 34 34
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
  8. 108 108
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java
  9. 4 4
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
  10. 38 38
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
  11. 6 276
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
  12. 43 43
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
  13. 42 42
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
  14. 123 1725
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
  15. 75 75
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftMetaStore.java
  16. 8 8
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java
  17. 108 108
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java
  18. 108 108
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java
  19. 4 4
      src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java
  20. 100 288
      src/contrib/hive/metastore/src/gen-php/ThriftHiveMetastore.php
  21. 153 153
      src/contrib/hive/metastore/src/gen-php/ThriftMetaStore.php
  22. 0 51
      src/contrib/hive/metastore/src/gen-php/hive_metastore_constants.php
  23. 96 432
      src/contrib/hive/metastore/src/gen-php/hive_metastore_types.php
  24. 6 27
      src/contrib/hive/metastore/src/gen-py/hive_metastore/ThriftHiveMetastore-remote
  25. 104 801
      src/contrib/hive/metastore/src/gen-py/hive_metastore/ThriftHiveMetastore.py
  26. 55 55
      src/contrib/hive/metastore/src/gen-py/hive_metastore/ThriftMetaStore.py
  27. 0 51
      src/contrib/hive/metastore/src/gen-py/hive_metastore/constants.py
  28. 186 345
      src/contrib/hive/metastore/src/gen-py/hive_metastore/ttypes.py
  29. 9 2
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/FileStore.java
  30. 55 125
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
  31. 8 73
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
  32. 3 1
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
  33. 10 3
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreClient.java
  34. 32 84
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreServer.java
  35. 118 53
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
  36. 57 44
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
  37. 2 0
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
  38. 1 1
      src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
  39. 9 8
      src/contrib/hive/metastore/src/model/org/apache/hadoop/hive/metastore/model/MDatabase.java
  40. 1 105
      src/contrib/hive/metastore/src/model/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java
  41. 78 45
      src/contrib/hive/metastore/src/model/package.jdo
  42. 1 1
      src/contrib/hive/metastore/src/test/org/apache/hadoop/hive/metastore/MetaStoreTestBase.java
  43. 26 22
      src/contrib/hive/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
  44. 10 17
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/Context.java
  45. 57 91
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
  46. 28 10
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/CollectOperator.java
  47. 3 2
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java
  48. 0 178
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/CompositeHiveObject.java
  49. 74 29
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
  50. 2 9
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java
  51. 103 28
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecReducer.java
  52. 7 4
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
  53. 42 8
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeColumnEvaluator.java
  54. 15 5
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantEvaluator.java
  55. 12 7
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluator.java
  56. 40 10
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeFieldEvaluator.java
  57. 42 20
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeFuncEvaluator.java
  58. 20 13
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeIndexEvaluator.java
  59. 9 4
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeNullEvaluator.java
  60. 6 2
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExtractOperator.java
  61. 32 14
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
  62. 12 9
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java
  63. 5 2
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ForwardOperator.java
  64. 2 2
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
  65. 67 41
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
  66. 0 97
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/HiveObject.java
  67. 68 58
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
  68. 0 60
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/LabeledCompositeHiveObject.java
  69. 56 30
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
  70. 1 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java
  71. 0 57
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/NullHiveObject.java
  72. 17 17
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
  73. 0 67
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/PrimitiveHiveObject.java
  74. 90 38
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
  75. 42 38
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
  76. 42 10
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java
  77. 0 167
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/TableHiveObject.java
  78. 0 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/UDAF.java
  79. 0 2
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/UDF.java
  80. 76 21
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
  81. 0 96
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/HiveObjectComparator.java
  82. 0 43
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/HiveObjectSerializer.java
  83. 0 220
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NaiiveJSONSerializer.java
  84. 0 253
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NaiiveSerializer.java
  85. 0 28
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NoTagHiveObjectComparator.java
  86. 0 64
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NoTagWritableComparableHiveObject.java
  87. 0 82
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NoTagWritableHiveObject.java
  88. 0 206
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/WritableComparableHiveObject.java
  89. 0 149
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/WritableHiveObject.java
  90. 24 10
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
  91. 0 132
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
  92. 28 5
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
  93. 114 157
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
  94. 35 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
  95. 28 13
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
  96. 2 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
  97. 46 10
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
  98. 3 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/InputSignature.java
  99. 39 18
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/PartitionPruner.java
  100. 4 24
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java

+ 3 - 0
CHANGES.txt

@@ -210,6 +210,9 @@ Trunk (unreleased changes)
 
   IMPROVEMENTS
 
+    HADOOP-4205. hive: metastore and ql to use the refactored SerDe library.
+    (zshao)
+
     HADOOP-4106. libhdfs: add time, permission and user attribute support (part 2).
     (Pete Wyckoff through zshao)
 

+ 23 - 22
src/contrib/hive/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java

@@ -82,30 +82,27 @@ public class CliDriver {
         e.printStackTrace();
       }
 
-    } else {
+    } 
+    else {
       ret = qp.run(cmd);
       Vector<Vector<String>> res = new Vector<Vector<String>>();
-      while (qp.getResults(res))
-      {
-        SessionState ss = SessionState.get();
-        OutputStream out = ss.out;
-        try 
-        {
+      while (qp.getResults(res)) {
+        SessionState ss  = SessionState.get();
+        PrintStream out = ss.out;
 
-          for (Vector<String> row:res)
+        for (Vector<String> row:res)
+        {
+          boolean firstCol = true;
+          for (String col:row)
           {
-            for (String col:row)
-            {
-              out.write(col == null ? Utilities.nullStringOutput.getBytes() : col.getBytes());
+            if (!firstCol)
               out.write(Utilities.tabCode);
-            } 
-            out.write(Utilities.newLineCode);
-          }
-          res.clear();
-
-        } catch (IOException e) {
-          e.printStackTrace();
+            out.print(col == null ? Utilities.nullStringOutput : col);
+            firstCol = false;
+          } 
+          out.write(Utilities.newLineCode);
         }
+        res.clear();
       }
     }
     return ret;
@@ -151,16 +148,20 @@ public class CliDriver {
     SessionState.initHiveLog4j();
 
     CliSessionState ss = new CliSessionState (new HiveConf(SessionState.class));
+    ss.in = System.in;
+    try {
+      ss.out = new PrintStream(System.out, true, "UTF-8");
+      ss.err = new PrintStream(System.err, true, "UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      System.exit(3);
+    }
+
     SessionState.start(ss);
 
     if(! oproc.process_stage2(ss)) {
       System.exit(2);
     }
 
-    ss.in = System.in;
-    ss.out = System.out;
-    ss.err = System.err;
-
     sp = new SetProcessor();
     qp = new Driver();
 

+ 138 - 185
src/contrib/hive/metastore/if/hive_metastore.thrift

@@ -1,172 +1,107 @@
 #!/usr/local/bin/thrift -java
 #
 # Thrift Service that the MetaStore is built on
-# Pete Wyckoff (pwyckoff@facebook.com)
 #
 
-/**
- * The available types in Thrift:
- *
- *  bool        Boolean, one byte
- *  byte        Signed byte
- *  i16         Signed 16-bit integer
- *  i32         Signed 32-bit integer
- *  i64         Signed 64-bit integer
- *  double      64-bit floating point value
- *  string      String
- *  map<t1,t2>  Map from one type to another
- *  list<t1>    Ordered list of one type
- *  set<t1>     Set of unique elements of one type
- *
- */
-
 include "thrift/fb303/if/fb303.thrift"
 
-
 namespace java org.apache.hadoop.hive.metastore.api
-namespace php metastore
-
-// below is terrible - tight, tight integration like you know who... bugbug
-// need to add a level of indirection to name these things..
-const string META_SERDE = "com.facebook.serde.simple_meta.MetadataTypedColumnsetSerDe"
-
-const string META_TABLE_COLUMNS  = "columns",
-const string BUCKET_FIELD_NAME   = "bucket_field_name",
-const string BUCKET_COUNT        = "bucket_count",
-const string FIELD_TO_DIMENSION  = "field_to_dimension",
-const string META_TABLE_NAME     = "name",
-const string META_TABLE_DB     = "db",
-const string META_TABLE_LOCATION = "location",
-const string META_TABLE_SERDE    = "serde",
-const string SERIALIZATION_FORMAT = "serialization.format",
-const string SERIALIZATION_CLASS = "serialization.class",
-const string SERIALIZATION_LIB = "serialization.lib",
-const string META_TABLE_PARTITION_COLUMNS = "partition_columns",
-const string FILE_INPUT_FORMAT = "file.inputformat",
-const string FILE_OUTPUT_FORMAT = "file.outputformat",
-
-
-exception MetaException {
-  string message
-}
-
-exception UnknownTableException {
-  string message
-}
-
-exception UnknownDBException {
-  string message
-}
-
-// new metastore api, below will be merged with above
-
-const string KEY_COMMENTS = "key_comments";
-const string VERSION_0_1 = "0.1";
-
-typedef string PrimitiveType
-typedef string CollectionType
-
-const string TINYINT_TYPE_NAME = "tinyint";
-const string INT_TYPE_NAME = "int";
-const string BIGINT_TYPE_NAME = "bigint";
-const string FLOAT_TYPE_NAME = "float";
-const string DOUBLE_TYPE_NAME = "double"; 
-const string STRING_TYPE_NAME = "string";
-const string DATE_TYPE_NAME = "date";
-const string DATETIME_TYPE_NAME = "datetime";
-const string TIMESTAMP_TYPE_NAME = "timestamp";
-
-const string LIST_TYPE_NAME = "list";
-const string MAP_TYPE_NAME = "map";
-
-const set<string> PrimitiveTypes = [ TINYINT_TYPE_NAME INT_TYPE_NAME BIGINT_TYPE_NAME FLOAT_TYPE_NAME DOUBLE_TYPE_NAME STRING_TYPE_NAME  DATE_TYPE_NAME DATETIME_TYPE_NAME TIMESTAMP_TYPE_NAME ],
-const set<string> CollectionTypes = [ LIST_TYPE_NAME MAP_TYPE_NAME ],
+php_namespace metastore
 
 struct Version {
-  string version,
-  string comments
+  1: string version,
+  2: string comments
 }
 
 struct FieldSchema {
-  string name, // name of the field
-  string type, // type of the field. primitive types defined above, specify list<TYPE_NAME>, map<TYPE_NAME, TYPE_NAME> for lists & maps 
-  string comment
+  1: string name, // name of the field
+  2: string type, // type of the field. primitive types defined above, specify list<TYPE_NAME>, map<TYPE_NAME, TYPE_NAME> for lists & maps 
+  3: string comment
 }
 
 struct Type {
-  string name, // one of the types in PrimitiveTypes or CollectionTypes or User defined types
-  optional string type1, // object type if the name is 'list' (LIST_TYPE), key type if the name is 'map' (MAP_TYPE)
-  optional string type2, // val type if the name is 'map' (MAP_TYPE)
-  optional list<FieldSchema> fields //if the name is one of the user defined types
+  1: string          name,             // one of the types in PrimitiveTypes or CollectionTypes or User defined types
+  2: optional string type1,            // object type if the name is 'list' (LIST_TYPE), key type if the name is 'map' (MAP_TYPE)
+  3: optional string type2,            // val type if the name is 'map' (MAP_TYPE)
+  4: optional list<FieldSchema> fields // if the name is one of the user defined types
 }
 
-// groups a set of tables
+// namespace for tables
 struct Database {
-  string name,
-  string locationUri,
+  1: string name,
+  2: string description,
 }
 
+// This object holds the information needed by SerDes
 struct SerDeInfo {
-  string name;
-  string serializationFormat;
-  string serializationClass;
-  string serializationLib;
-  string fieldDelim;
-  string collectionItemDelim;
-  string mapKeyDelim;
-  string lineDelim;
-  map<string, string> parameters
+  1: string name,                   // name of the serde, table name by default
+  2: string serializationLib,       // usually the class that implements the extractor & loader
+  3: map<string, string> parameters // initialization parameters
 }
 
+// sort order of a column (column name along with asc(1)/desc(0))
 struct Order {
-  string col,
-  i32 order
+  1: string col,  // sort column name
+  2: i32    order // asc(1) or desc(0)
 }
 
+// this object holds all the information about physical storage of the data belonging to a table
 struct StorageDescriptor {
-  list<FieldSchema> cols, // required (refer to types defined above)
-  string location, // defaults to <warehouse loc>/<db loc>/tablename
-  string inputFormat; // SequenceFileInputFormat (binary) or TextInputFormat`  or custom format
-  string outputFormat; // SequenceFileOutputFormat (binary) or IgnoreKeyTextOutputFormat or custom format
-  bool isCompressed; // compressed or not
-  i32 numBuckets = 32, // this must be specified if there are any dimension columns
-  SerDeInfo serdeInfo; // serialization and deserialization information
-  list<string> bucketCols, // reducer grouping columns and clustering columns and bucketing columns`
-  list<Order> sortCols, // sort order of the data in each bucket
-  map<string, string> parameters // any user supplied key value hash
-}
-
+  1: list<FieldSchema> cols,  // required (refer to types defined above)
+  2: string location,         // defaults to <warehouse loc>/<db loc>/tablename
+  3: string inputFormat,      // SequenceFileInputFormat (binary) or TextInputFormat`  or custom format
+  4: string outputFormat,     // SequenceFileOutputFormat (binary) or IgnoreKeyTextOutputFormat or custom format
+  5: bool   compressed,       // compressed or not
+  6: i32    numBuckets,       // this must be specified if there are any dimension columns
+  7: SerDeInfo    serdeInfo,  // serialization and deserialization information
+  8: list<string> bucketCols, // reducer grouping columns and clustering columns and bucketing columns`
+  9: list<Order>  sortCols,   // sort order of the data in each bucket
+  10: map<string, string> parameters // any user supplied key value hash
+}
+
+// table information
 struct Table {
-  string tableName, 
-  string database,
-  string owner,
-  i32 createTime,
-  i32 lastAccessTime,
-  i32 retention,
-  StorageDescriptor sd,
-  list<FieldSchema> partitionKeys, // optional
-  map<string, string> parameters // to store comments or any other user level parameters
+  1: string tableName,                // name of the table
+  2: string dbName,                   // database name ('default')
+  3: string owner,                    // owner of this table
+  4: i32    createTime,               // creation time of the table
+  5: i32    lastAccessTime,           // last access time (usually this will be filled from HDFS and shouldn't be relied on)
+  6: i32    retention,                // retention time
+  7: StorageDescriptor sd,            // storage descriptor of the table
+  8: list<FieldSchema> partitionKeys, // partition keys of the table. only primitive types are supported
+  9: map<string, string> parameters   // to store comments or any other user level parameters
 }
 
 struct Partition {
-  // keys are inherited from table. this should be okay because partition keys can't be changed over time
-  list<string> values // string value is converted to appropriate partition key type
-  string database,
-  string tableName,
-  i32 createTime,
-  i32 lastAccessTime,
-  StorageDescriptor sd,
-  map<string, string> parameters
+  1: list<string> values // string value is converted to appropriate partition key type
+  2: string       dbName,
+  3: string       tableName,
+  4: i32          createTime,
+  5: i32          lastAccessTime,
+  6: StorageDescriptor   sd,
+  7: map<string, string> parameters
 }
 
 // index on a hive table is also another table whose columns are the subset of the base table columns along with the offset
 // this will automatically generate table (table_name_index_name)
 struct Index {
-  string indexName, // unique with in the whole database namespace
-  i32 indexType, // reserved
-  string tableName,
-  string databaseName,
-  list<string> colNames, // for now columns will be sorted in the ascending order
+  1: string       indexName, // unique with in the whole database namespace
+  2: i32          indexType, // reserved
+  3: string       tableName,
+  4: string       dbName,
+  5: list<string> colNames,  // for now columns will be sorted in the ascending order
+}
+
+exception MetaException {
+  string message
+}
+
+exception UnknownTableException {
+  string message
+}
+
+exception UnknownDBException {
+  string message
 }
 
 exception AlreadyExistsException {
@@ -194,105 +129,123 @@ exception InvalidOperationException {
 }
 
 /**
-* This interface is NOT live yet.
+* This interface is live.
 */
 service ThriftHiveMetastore extends fb303.FacebookService
 {
-  // Database
-  bool create_database(1:string name, 2:string location_uri) throws(1:AlreadyExistsException o1, 2:MetaException o2)
+  bool create_database(1:string name, 2:string description) 
+                                       throws(1:AlreadyExistsException o1, 2:MetaException o2)
   Database get_database(1:string name) throws(1:NoSuchObjectException o1, 2:MetaException o2)
-  bool drop_database(1:string name) throws(2:MetaException o2)
-  list<string> get_databases() throws(1:MetaException o1)
+  bool drop_database(1:string name)    throws(2:MetaException o2)
+  list<string> get_databases()         throws(1:MetaException o1)
 
-  // Type
   // returns the type with given name (make seperate calls for the dependent types if needed)
-  Type get_type(1:string name) throws(1:MetaException o2)
+  Type get_type(1:string name)  throws(1:MetaException o2)
   bool create_type(1:Type type) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3)
   bool drop_type(1:string type) throws(1:MetaException o2)
-  map<string, Type> get_type_all(1:string name) throws(1:MetaException o2)
+  map<string, Type> get_type_all(1:string name) 
+                                throws(1:MetaException o2)
 
-  list<FieldSchema> get_fields(string db_name, string table_name) throws (MetaException ouch1, UnknownTableException ouch2, UnknownDBException ouch3),
+  list<FieldSchema> get_fields(string db_name, string table_name) throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
 
-  // Tables
   // create a Hive table. Following fields must be set
-  // Table.tableName
-  // Table.database (only 'default' for now until Hive QL supports databases)
-  // Table.owner (not needed, but good to have for tracking purposes)
-  // Table.sd.cols (list of field schemas)
-  // Table.sd.inputFormat ( SequenceFileInputFormat (binary like falcon tables or u_full) or TextInputFormat)
-  // Table.sd.outputFormat ( SequenceFileInputFormat (binary) or TextInputFormat)
-  // Table.sd.serdeInfo.serializationLib (SerDe class name such as org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe
-  void create_table(1:Table tbl) throws(1:AlreadyExistsException ouch1, 2:InvalidObjectException ouch2, 3:MetaException ouch3, 4:NoSuchObjectException o4)
+  // tableName
+  // database        (only 'default' for now until Hive QL supports databases)
+  // owner           (not needed, but good to have for tracking purposes)
+  // sd.cols         (list of field schemas)
+  // sd.inputFormat  (SequenceFileInputFormat (binary like falcon tables or u_full) or TextInputFormat)
+  // sd.outputFormat (SequenceFileInputFormat (binary) or TextInputFormat)
+  // sd.serdeInfo.serializationLib (SerDe class name eg org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe
+  void create_table(1:Table tbl) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3, 4:NoSuchObjectException o4)
   // drops the table and all the partitions associated with it if the table has partitions
   // delete data (including partitions) if deleteData is set to true
-  void drop_table(1:string dbname, 2:string name, 3:bool deleteData) throws(1:NoSuchObjectException o1, 2:MetaException ouch3)
-  list<string> get_tables(string db_name, string pattern)  throws (MetaException ouch1, UnknownTableException ouch2, UnknownDBException ouch3)
-  Table get_table(1:string dbname, 2:string tbl_name) throws (1:MetaException o1, 2:NoSuchObjectException o2)
-  bool set_table_parameters(1:string dbname, 2:string tbl_name, 3:map<string, string> params) throws(1:NoSuchObjectException o1, 2:MetaException o2)
-  // this applies to only future partitions not for existing partitions
-  void alter_table(1:string dbname, 2:string tbl_name, 3:Table new_tbl) throws (1:InvalidOperationException o1, 2:MetaException o2)
-  void truncate_table(1:string db_name, 2:string table_name, 3:string partition)  throws (1:MetaException ouch1, 2:UnknownTableException ouch2, 3:UnknownDBException ouch3),
-  list<string> cat(1:string db_name, 2:string table_name, 3:string partition, i32 high) throws  (MetaException ouch1, UnknownDBException ouch2, UnknownTableException ouch3),
-
-  // Partition
+  void drop_table(1:string dbname, 2:string name, 3:bool deleteData) 
+                       throws(1:NoSuchObjectException o1, 2:MetaException o3)
+  list<string> get_tables(string db_name, string pattern)
+                       throws (MetaException o1, UnknownTableException o2, UnknownDBException o3)
+  Table get_table(1:string dbname, 2:string tbl_name) 
+                       throws (1:MetaException o1, 2:NoSuchObjectException o2)
+  // alter table applies to only future partitions not for existing partitions
+  void alter_table(1:string dbname, 2:string tbl_name, 3:Table new_tbl) 
+                       throws (1:InvalidOperationException o1, 2:MetaException o2)
+
   // the following applies to only tables that have partitions
-  Partition add_partition(1:Partition new_part) throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
-  Partition append_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals) throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
-  bool drop_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals, 4:bool deleteData) throws(1:NoSuchObjectException o1, 2:MetaException o2)
-  Partition get_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals) throws(1:MetaException o1)
-  // returns all the partitions for this table in reverse chronological order. if max parts is given then it will return only that many
-  list<Partition> get_partitions(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1) throws(1:NoSuchObjectException o1, 2:MetaException o2)
-  // this can be used if an old parition has to be recreated
-  bool set_partition_parameters(1:string db_name, 2:string tbl_name, 3:string pname, 4:map<string, string> params) throws(1:NoSuchObjectException o1, 2:MetaException o2)
-  // changes the given partitions to the new storage descriptor. all partitions should belong to the same table
-  bool alter_partitions(1:StorageDescriptor sd, 2:list<string> parts) throws(1:InvalidOperationException o1, 2:MetaException o2)
-
-  // Index
+  Partition add_partition(1:Partition new_part) 
+                       throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  Partition append_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals) 
+                       throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3)
+  bool drop_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals, 4:bool deleteData) 
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  Partition get_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals) 
+                       throws(1:MetaException o1)
+  // returns all the partitions for this table in reverse chronological order. 
+  // if max parts is given then it will return only that many
+  list<Partition> get_partitions(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1) 
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  list<string> get_partition_names(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1) 
+                       throws(1:MetaException o2)
+  // set new storage descriptor. all partitions should belong to the same table
+  bool alter_partitions(1:StorageDescriptor sd, 2:list<string> parts) 
+                       throws(1:InvalidOperationException o1, 2:MetaException o2)
+
   // index related metadata (may not be in the open source)
   bool create_index(1:Index index_def) throws(1:IndexAlreadyExistsException o1, 2:MetaException o2)
 }
 
 
 /**
-* This interface is the live.
+* This interface is deprecated.
 */
 service ThriftMetaStore extends fb303.FacebookService
 {
   // retrieve a printable representation of the fields in a table (logfile, type) or table subtype
-  list<FieldSchema> get_fields(string db_name, string table_name) throws (MetaException ouch1, UnknownTableException ouch2, UnknownDBException ouch3),
+  list<FieldSchema> get_fields(string db_name, string table_name) throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
 
   // get all the tables (logfiles, types) in the metastore - no partitioning like different dbs yet
-  list<string> get_tables(string db_name, string pattern)  throws (MetaException ouch1, UnknownTableException ouch2, UnknownDBException ouch3),
+  list<string> get_tables(string db_name, string pattern)  throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
 
   // retrieve the opaque schema representation of this table (logfile, type) which contains enough
   // information for the caller to instantiate some kind of object that will let it examine the type.
   // That object might be a thrift, jute, or SerDe.
-  map<string,string> get_schema(string table_name) throws (MetaException ouch1, UnknownTableException ouch2, UnknownDBException ouch3),
+  map<string,string> get_schema(string table_name) throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
 
   // add some structure to the table or change its structure
-  void alter_table(string db_name, string table_name, map<string,string> schema) throws (MetaException ouch1, UnknownTableException ouch2, UnknownDBException ouch3),
+  void alter_table(string db_name, string table_name, map<string,string> schema) throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
 
   // create_table == create_table4 (table_name, SIMPLE_META_SERDE, '\t', "",  dict [ META_COLUMNS => columns]
   // bugbug do above transformation and deprecate this API
-  void create_table(string db_name, string table_name, map<string,string> schema) throws (MetaException ouch1, UnknownDBException ouch2),
+  void create_table(string db_name, string table_name, map<string,string> schema) throws (MetaException o1, UnknownDBException o2),
 
   // drop a table (i.e., remove it from the metastore) - for now allow metastore to do the delete (so python shell can do drops)
-  void drop_table(string db_name, string table_name) throws  (MetaException ouch1, UnknownTableException ouch2, UnknownDBException ouch3),
+  void drop_table(string db_name, string table_name) throws  (MetaException o1, UnknownTableException o2, UnknownDBException o3),
 
   // truncate a table - i.e., delete its data, but keep the hdfs directory and the schema
-  void truncate_table(string db_name, string table_name, string partition)  throws (MetaException ouch1, UnknownTableException ouch2, UnknownDBException ouch3),
+  void truncate_table(string db_name, string table_name, string partition)  throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
 
   // generally does the table exist
-  bool table_exists(string db_name, string table_name) throws (MetaException ouch1, UnknownDBException ouch2),
+  bool table_exists(string db_name, string table_name) throws (MetaException o1, UnknownDBException o2),
 
   // create a table with named columns
-  list<string> get_partitions(string db_name, string table_name) throws (MetaException ouch1, UnknownTableException ouch2, UnknownDBException ouch3),
+  list<string> get_partitions(string db_name, string table_name) throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
 
   // enumerate all the databases in this store
-  list<string> get_dbs() throws  (MetaException ouch),
+  list<string> get_dbs() throws  (MetaException o),
 
   // /bin/cat the table in human readable format
-  list<string> cat(string db_name, string table_name,string partition, i32 high) throws  (MetaException ouch1, UnknownDBException ouch2, UnknownTableException ouch3),
-}
+  list<string> cat(string db_name, string table_name,string partition, i32 high) throws  (MetaException o1, UnknownDBException o2, UnknownTableException o3),
+}
+
+// these should be needed only for backward compatibility with filestore
+const string META_TABLE_COLUMNS   = "columns",
+const string BUCKET_FIELD_NAME    = "bucket_field_name",
+const string BUCKET_COUNT         = "bucket_count",
+const string FIELD_TO_DIMENSION   = "field_to_dimension",
+const string META_TABLE_NAME      = "name",
+const string META_TABLE_DB        = "db",
+const string META_TABLE_LOCATION  = "location",
+const string META_TABLE_SERDE     = "serde",
+const string META_TABLE_PARTITION_COLUMNS = "partition_columns",
+const string FILE_INPUT_FORMAT    = "file.inputformat",
+const string FILE_OUTPUT_FORMAT   = "file.outputformat",
 
 

+ 0 - 53
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Constants.java

@@ -15,8 +15,6 @@ import com.facebook.thrift.*;
 
 public class Constants {
 
-  public static final String META_SERDE = "com.facebook.serde.simple_meta.MetadataTypedColumnsetSerDe";
-
   public static final String META_TABLE_COLUMNS = "columns";
 
   public static final String BUCKET_FIELD_NAME = "bucket_field_name";
@@ -33,61 +31,10 @@ public class Constants {
 
   public static final String META_TABLE_SERDE = "serde";
 
-  public static final String SERIALIZATION_FORMAT = "serialization.format";
-
-  public static final String SERIALIZATION_CLASS = "serialization.class";
-
-  public static final String SERIALIZATION_LIB = "serialization.lib";
-
   public static final String META_TABLE_PARTITION_COLUMNS = "partition_columns";
 
   public static final String FILE_INPUT_FORMAT = "file.inputformat";
 
   public static final String FILE_OUTPUT_FORMAT = "file.outputformat";
 
-  public static final String KEY_COMMENTS = "key_comments";
-
-  public static final String VERSION_0_1 = "0.1";
-
-  public static final String TINYINT_TYPE_NAME = "tinyint";
-
-  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 DATE_TYPE_NAME = "date";
-
-  public static final String DATETIME_TYPE_NAME = "datetime";
-
-  public static final String TIMESTAMP_TYPE_NAME = "timestamp";
-
-  public static final String LIST_TYPE_NAME = "list";
-
-  public static final String MAP_TYPE_NAME = "map";
-
-  public static final Set<String> PrimitiveTypes = new HashSet<String>();
-  static {
-    PrimitiveTypes.add("tinyint");
-    PrimitiveTypes.add("int");
-    PrimitiveTypes.add("bigint");
-    PrimitiveTypes.add("float");
-    PrimitiveTypes.add("double");
-    PrimitiveTypes.add("string");
-    PrimitiveTypes.add("date");
-    PrimitiveTypes.add("datetime");
-    PrimitiveTypes.add("timestamp");
-  }
-
-  public static final Set<String> CollectionTypes = new HashSet<String>();
-  static {
-    CollectionTypes.add("list");
-    CollectionTypes.add("map");
-  }
-
 }

+ 28 - 28
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java

@@ -18,12 +18,12 @@ import com.facebook.thrift.transport.*;
 
 public class Database implements TBase, java.io.Serializable {
 private String name;
-private String locationUri;
+private String description;
 
 public final Isset __isset = new Isset();
 public static final class Isset implements java.io.Serializable {
 public boolean name = false;
-public boolean locationUri = false;
+public boolean description = false;
 }
 
 public Database() {
@@ -31,13 +31,13 @@ public Database() {
 
 public Database(
 String name,
-String locationUri)
+String description)
 {
 this();
 this.name = name;
 this.__isset.name = true;
-this.locationUri = locationUri;
-this.__isset.locationUri = true;
+this.description = description;
+this.__isset.description = true;
 }
 
 public String getName() {
@@ -53,17 +53,17 @@ public void unsetName() {
 this.__isset.name = false;
 }
 
-public String getLocationUri() {
-return this.locationUri;
+public String getDescription() {
+return this.description;
 }
 
-public void setLocationUri(String locationUri) {
-this.locationUri = locationUri;
-this.__isset.locationUri = true;
+public void setDescription(String description) {
+this.description = description;
+this.__isset.description = true;
 }
 
-public void unsetLocationUri() {
-this.__isset.locationUri = false;
+public void unsetDescription() {
+this.__isset.description = false;
 }
 
 public boolean equals(Object that) {
@@ -87,12 +87,12 @@ if (!this.name.equals(that.name))
   return false;
 }
 
-boolean this_present_locationUri = true && (this.locationUri != null);
-boolean that_present_locationUri = true && (that.locationUri != null);
-if (this_present_locationUri || that_present_locationUri) {
-if (!(this_present_locationUri && that_present_locationUri))
+boolean this_present_description = true && (this.description != null);
+boolean that_present_description = true && (that.description != null);
+if (this_present_description || that_present_description) {
+if (!(this_present_description && that_present_description))
   return false;
-if (!this.locationUri.equals(that.locationUri))
+if (!this.description.equals(that.description))
   return false;
 }
 
@@ -114,7 +114,7 @@ if (field.type == TType.STOP) {
 }
 switch (field.id)
 {
-  case -1:
+  case 1:
     if (field.type == TType.STRING) {
       this.name = iprot.readString();
       this.__isset.name = true;
@@ -122,10 +122,10 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -2:
+  case 2:
     if (field.type == TType.STRING) {
-      this.locationUri = iprot.readString();
-      this.__isset.locationUri = true;
+      this.description = iprot.readString();
+      this.__isset.description = true;
     } else { 
       TProtocolUtil.skip(iprot, field.type);
     }
@@ -146,17 +146,17 @@ TField field = new TField();
 if (this.name != null) {
 field.name = "name";
 field.type = TType.STRING;
-field.id = -1;
+field.id = 1;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.name);
 oprot.writeFieldEnd();
 }
-if (this.locationUri != null) {
-field.name = "locationUri";
+if (this.description != null) {
+field.name = "description";
 field.type = TType.STRING;
-field.id = -2;
+field.id = 2;
 oprot.writeFieldBegin(field);
-oprot.writeString(this.locationUri);
+oprot.writeString(this.description);
 oprot.writeFieldEnd();
 }
 oprot.writeFieldStop();
@@ -167,8 +167,8 @@ public String toString() {
 StringBuilder sb = new StringBuilder("Database(");
 sb.append("name:");
 sb.append(this.name);
-sb.append(",locationUri:");
-sb.append(this.locationUri);
+sb.append(",description:");
+sb.append(this.description);
 sb.append(")");
 return sb.toString();
 }

+ 6 - 6
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/FieldSchema.java

@@ -141,7 +141,7 @@ public class FieldSchema implements TBase, java.io.Serializable {
       }
       switch (field.id)
       {
-        case -1:
+        case 1:
           if (field.type == TType.STRING) {
             this.name = iprot.readString();
             this.__isset.name = true;
@@ -149,7 +149,7 @@ public class FieldSchema implements TBase, java.io.Serializable {
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case -2:
+        case 2:
           if (field.type == TType.STRING) {
             this.type = iprot.readString();
             this.__isset.type = true;
@@ -157,7 +157,7 @@ public class FieldSchema implements TBase, java.io.Serializable {
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case -3:
+        case 3:
           if (field.type == TType.STRING) {
             this.comment = iprot.readString();
             this.__isset.comment = true;
@@ -181,7 +181,7 @@ public class FieldSchema implements TBase, java.io.Serializable {
     if (this.name != null) {
       field.name = "name";
       field.type = TType.STRING;
-      field.id = -1;
+      field.id = 1;
       oprot.writeFieldBegin(field);
       oprot.writeString(this.name);
       oprot.writeFieldEnd();
@@ -189,7 +189,7 @@ public class FieldSchema implements TBase, java.io.Serializable {
     if (this.type != null) {
       field.name = "type";
       field.type = TType.STRING;
-      field.id = -2;
+      field.id = 2;
       oprot.writeFieldBegin(field);
       oprot.writeString(this.type);
       oprot.writeFieldEnd();
@@ -197,7 +197,7 @@ public class FieldSchema implements TBase, java.io.Serializable {
     if (this.comment != null) {
       field.name = "comment";
       field.type = TType.STRING;
-      field.id = -3;
+      field.id = 3;
       oprot.writeFieldBegin(field);
       oprot.writeString(this.comment);
       oprot.writeFieldEnd();

+ 34 - 34
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java

@@ -20,7 +20,7 @@ public class Index implements TBase, java.io.Serializable {
 private String indexName;
 private int indexType;
 private String tableName;
-private String databaseName;
+private String dbName;
 private List<String> colNames;
 
 public final Isset __isset = new Isset();
@@ -28,7 +28,7 @@ public static final class Isset implements java.io.Serializable {
 public boolean indexName = false;
 public boolean indexType = false;
 public boolean tableName = false;
-public boolean databaseName = false;
+public boolean dbName = false;
 public boolean colNames = false;
 }
 
@@ -39,7 +39,7 @@ public Index(
 String indexName,
 int indexType,
 String tableName,
-String databaseName,
+String dbName,
 List<String> colNames)
 {
 this();
@@ -49,8 +49,8 @@ this.indexType = indexType;
 this.__isset.indexType = true;
 this.tableName = tableName;
 this.__isset.tableName = true;
-this.databaseName = databaseName;
-this.__isset.databaseName = true;
+this.dbName = dbName;
+this.__isset.dbName = true;
 this.colNames = colNames;
 this.__isset.colNames = true;
 }
@@ -94,17 +94,17 @@ public void unsetTableName() {
 this.__isset.tableName = false;
 }
 
-public String getDatabaseName() {
-return this.databaseName;
+public String getDbName() {
+return this.dbName;
 }
 
-public void setDatabaseName(String databaseName) {
-this.databaseName = databaseName;
-this.__isset.databaseName = true;
+public void setDbName(String dbName) {
+this.dbName = dbName;
+this.__isset.dbName = true;
 }
 
-public void unsetDatabaseName() {
-this.__isset.databaseName = false;
+public void unsetDbName() {
+this.__isset.dbName = false;
 }
 
 public int getColNamesSize() {
@@ -176,12 +176,12 @@ if (!this.tableName.equals(that.tableName))
   return false;
 }
 
-boolean this_present_databaseName = true && (this.databaseName != null);
-boolean that_present_databaseName = true && (that.databaseName != null);
-if (this_present_databaseName || that_present_databaseName) {
-if (!(this_present_databaseName && that_present_databaseName))
+boolean this_present_dbName = true && (this.dbName != null);
+boolean that_present_dbName = true && (that.dbName != null);
+if (this_present_dbName || that_present_dbName) {
+if (!(this_present_dbName && that_present_dbName))
   return false;
-if (!this.databaseName.equals(that.databaseName))
+if (!this.dbName.equals(that.dbName))
   return false;
 }
 
@@ -212,7 +212,7 @@ if (field.type == TType.STOP) {
 }
 switch (field.id)
 {
-  case -1:
+  case 1:
     if (field.type == TType.STRING) {
       this.indexName = iprot.readString();
       this.__isset.indexName = true;
@@ -220,7 +220,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -2:
+  case 2:
     if (field.type == TType.I32) {
       this.indexType = iprot.readI32();
       this.__isset.indexType = true;
@@ -228,7 +228,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -3:
+  case 3:
     if (field.type == TType.STRING) {
       this.tableName = iprot.readString();
       this.__isset.tableName = true;
@@ -236,15 +236,15 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -4:
+  case 4:
     if (field.type == TType.STRING) {
-      this.databaseName = iprot.readString();
-      this.__isset.databaseName = true;
+      this.dbName = iprot.readString();
+      this.__isset.dbName = true;
     } else { 
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -5:
+  case 5:
     if (field.type == TType.LIST) {
       {
         TList _list44 = iprot.readListBegin();
@@ -278,37 +278,37 @@ TField field = new TField();
 if (this.indexName != null) {
 field.name = "indexName";
 field.type = TType.STRING;
-field.id = -1;
+field.id = 1;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.indexName);
 oprot.writeFieldEnd();
 }
 field.name = "indexType";
 field.type = TType.I32;
-field.id = -2;
+field.id = 2;
 oprot.writeFieldBegin(field);
 oprot.writeI32(this.indexType);
 oprot.writeFieldEnd();
 if (this.tableName != null) {
 field.name = "tableName";
 field.type = TType.STRING;
-field.id = -3;
+field.id = 3;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.tableName);
 oprot.writeFieldEnd();
 }
-if (this.databaseName != null) {
-field.name = "databaseName";
+if (this.dbName != null) {
+field.name = "dbName";
 field.type = TType.STRING;
-field.id = -4;
+field.id = 4;
 oprot.writeFieldBegin(field);
-oprot.writeString(this.databaseName);
+oprot.writeString(this.dbName);
 oprot.writeFieldEnd();
 }
 if (this.colNames != null) {
 field.name = "colNames";
 field.type = TType.LIST;
-field.id = -5;
+field.id = 5;
 oprot.writeFieldBegin(field);
 {
   oprot.writeListBegin(new TList(TType.STRING, this.colNames.size()));
@@ -331,8 +331,8 @@ sb.append(",indexType:");
 sb.append(this.indexType);
 sb.append(",tableName:");
 sb.append(this.tableName);
-sb.append(",databaseName:");
-sb.append(this.databaseName);
+sb.append(",dbName:");
+sb.append(this.dbName);
 sb.append(",colNames:");
 sb.append(this.colNames);
 sb.append(")");

+ 108 - 108
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/MetaException.java

@@ -17,116 +17,116 @@ import com.facebook.thrift.protocol.*;
 import com.facebook.thrift.transport.*;
 
 public class MetaException extends Exception implements TBase, java.io.Serializable {
-  private String message;
-
-  public final Isset __isset = new Isset();
-  public static final class Isset implements java.io.Serializable {
-    public boolean message = false;
-  }
-
-  public MetaException() {
-  }
-
-  public MetaException(
-    String message)
-  {
-    this();
-    this.message = message;
-    this.__isset.message = true;
-  }
-
-  public String getMessage() {
-    return this.message;
-  }
-
-  public void setMessage(String message) {
-    this.message = message;
-    this.__isset.message = true;
-  }
-
-  public void unsetMessage() {
-    this.__isset.message = false;
-  }
-
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof MetaException)
-      return this.equals((MetaException)that);
-    return false;
-  }
-
-  public boolean equals(MetaException that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_message = true && (this.message != null);
-    boolean that_present_message = true && (that.message != null);
-    if (this_present_message || that_present_message) {
-      if (!(this_present_message && that_present_message))
-        return false;
-      if (!this.message.equals(that.message))
-        return false;
-    }
+private String message;
 
-    return true;
-  }
-
-  public int hashCode() {
-    return 0;
-  }
-
-  public void read(TProtocol iprot) throws TException {
-    TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == TType.STOP) { 
-        break;
-      }
-      switch (field.id)
-      {
-        case -1:
-          if (field.type == TType.STRING) {
-            this.message = iprot.readString();
-            this.__isset.message = true;
-          } else { 
-            TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          TProtocolUtil.skip(iprot, field.type);
-          break;
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-  }
-
-  public void write(TProtocol oprot) throws TException {
-    TStruct struct = new TStruct("MetaException");
-    oprot.writeStructBegin(struct);
-    TField field = new TField();
-    if (this.message != null) {
-      field.name = "message";
-      field.type = TType.STRING;
-      field.id = -1;
-      oprot.writeFieldBegin(field);
-      oprot.writeString(this.message);
-      oprot.writeFieldEnd();
+public final Isset __isset = new Isset();
+public static final class Isset implements java.io.Serializable {
+public boolean message = false;
+}
+
+public MetaException() {
+}
+
+public MetaException(
+String message)
+{
+this();
+this.message = message;
+this.__isset.message = true;
+}
+
+public String getMessage() {
+return this.message;
+}
+
+public void setMessage(String message) {
+this.message = message;
+this.__isset.message = true;
+}
+
+public void unsetMessage() {
+this.__isset.message = false;
+}
+
+public boolean equals(Object that) {
+if (that == null)
+  return false;
+if (that instanceof MetaException)
+  return this.equals((MetaException)that);
+return false;
+}
+
+public boolean equals(MetaException that) {
+if (that == null)
+  return false;
+
+boolean this_present_message = true && (this.message != null);
+boolean that_present_message = true && (that.message != null);
+if (this_present_message || that_present_message) {
+if (!(this_present_message && that_present_message))
+  return false;
+if (!this.message.equals(that.message))
+  return false;
+}
+
+return true;
+}
+
+public int hashCode() {
+return 0;
+}
+
+public void read(TProtocol iprot) throws TException {
+TField field;
+iprot.readStructBegin();
+while (true)
+{
+field = iprot.readFieldBegin();
+if (field.type == TType.STOP) { 
+  break;
+}
+switch (field.id)
+{
+  case -1:
+    if (field.type == TType.STRING) {
+      this.message = iprot.readString();
+      this.__isset.message = true;
+    } else { 
+      TProtocolUtil.skip(iprot, field.type);
     }
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
-  }
-
-  public String toString() {
-    StringBuilder sb = new StringBuilder("MetaException(");
-    sb.append("message:");
-    sb.append(this.message);
-    sb.append(")");
-    return sb.toString();
-  }
+    break;
+  default:
+    TProtocolUtil.skip(iprot, field.type);
+    break;
+}
+iprot.readFieldEnd();
+}
+iprot.readStructEnd();
+}
+
+public void write(TProtocol oprot) throws TException {
+TStruct struct = new TStruct("MetaException");
+oprot.writeStructBegin(struct);
+TField field = new TField();
+if (this.message != null) {
+field.name = "message";
+field.type = TType.STRING;
+field.id = -1;
+oprot.writeFieldBegin(field);
+oprot.writeString(this.message);
+oprot.writeFieldEnd();
+}
+oprot.writeFieldStop();
+oprot.writeStructEnd();
+}
+
+public String toString() {
+StringBuilder sb = new StringBuilder("MetaException(");
+sb.append("message:");
+sb.append(this.message);
+sb.append(")");
+return sb.toString();
+}
 
 }
 

+ 4 - 4
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java

@@ -114,7 +114,7 @@ if (field.type == TType.STOP) {
 }
 switch (field.id)
 {
-  case -1:
+  case 1:
     if (field.type == TType.STRING) {
       this.col = iprot.readString();
       this.__isset.col = true;
@@ -122,7 +122,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -2:
+  case 2:
     if (field.type == TType.I32) {
       this.order = iprot.readI32();
       this.__isset.order = true;
@@ -146,14 +146,14 @@ TField field = new TField();
 if (this.col != null) {
 field.name = "col";
 field.type = TType.STRING;
-field.id = -1;
+field.id = 1;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.col);
 oprot.writeFieldEnd();
 }
 field.name = "order";
 field.type = TType.I32;
-field.id = -2;
+field.id = 2;
 oprot.writeFieldBegin(field);
 oprot.writeI32(this.order);
 oprot.writeFieldEnd();

+ 38 - 38
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java

@@ -18,7 +18,7 @@ import com.facebook.thrift.transport.*;
 
 public class Partition implements TBase, java.io.Serializable {
 private List<String> values;
-private String database;
+private String dbName;
 private String tableName;
 private int createTime;
 private int lastAccessTime;
@@ -28,7 +28,7 @@ private Map<String,String> parameters;
 public final Isset __isset = new Isset();
 public static final class Isset implements java.io.Serializable {
 public boolean values = false;
-public boolean database = false;
+public boolean dbName = false;
 public boolean tableName = false;
 public boolean createTime = false;
 public boolean lastAccessTime = false;
@@ -41,7 +41,7 @@ public Partition() {
 
 public Partition(
 List<String> values,
-String database,
+String dbName,
 String tableName,
 int createTime,
 int lastAccessTime,
@@ -51,8 +51,8 @@ Map<String,String> parameters)
 this();
 this.values = values;
 this.__isset.values = true;
-this.database = database;
-this.__isset.database = true;
+this.dbName = dbName;
+this.__isset.dbName = true;
 this.tableName = tableName;
 this.__isset.tableName = true;
 this.createTime = createTime;
@@ -95,17 +95,17 @@ this.values = null;
 this.__isset.values = false;
 }
 
-public String getDatabase() {
-return this.database;
+public String getDbName() {
+return this.dbName;
 }
 
-public void setDatabase(String database) {
-this.database = database;
-this.__isset.database = true;
+public void setDbName(String dbName) {
+this.dbName = dbName;
+this.__isset.dbName = true;
 }
 
-public void unsetDatabase() {
-this.__isset.database = false;
+public void unsetDbName() {
+this.__isset.dbName = false;
 }
 
 public String getTableName() {
@@ -208,12 +208,12 @@ if (!this.values.equals(that.values))
   return false;
 }
 
-boolean this_present_database = true && (this.database != null);
-boolean that_present_database = true && (that.database != null);
-if (this_present_database || that_present_database) {
-if (!(this_present_database && that_present_database))
+boolean this_present_dbName = true && (this.dbName != null);
+boolean that_present_dbName = true && (that.dbName != null);
+if (this_present_dbName || that_present_dbName) {
+if (!(this_present_dbName && that_present_dbName))
   return false;
-if (!this.database.equals(that.database))
+if (!this.dbName.equals(that.dbName))
   return false;
 }
 
@@ -280,7 +280,7 @@ if (field.type == TType.STOP) {
 }
 switch (field.id)
 {
-  case -1:
+  case 1:
     if (field.type == TType.LIST) {
       {
         TList _list35 = iprot.readListBegin();
@@ -298,15 +298,15 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -2:
+  case 2:
     if (field.type == TType.STRING) {
-      this.database = iprot.readString();
-      this.__isset.database = true;
+      this.dbName = iprot.readString();
+      this.__isset.dbName = true;
     } else { 
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -3:
+  case 3:
     if (field.type == TType.STRING) {
       this.tableName = iprot.readString();
       this.__isset.tableName = true;
@@ -314,7 +314,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -4:
+  case 4:
     if (field.type == TType.I32) {
       this.createTime = iprot.readI32();
       this.__isset.createTime = true;
@@ -322,7 +322,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -5:
+  case 5:
     if (field.type == TType.I32) {
       this.lastAccessTime = iprot.readI32();
       this.__isset.lastAccessTime = true;
@@ -330,7 +330,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -6:
+  case 6:
     if (field.type == TType.STRUCT) {
       this.sd = new StorageDescriptor();
       this.sd.read(iprot);
@@ -339,7 +339,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -7:
+  case 7:
     if (field.type == TType.MAP) {
       {
         TMap _map38 = iprot.readMapBegin();
@@ -375,7 +375,7 @@ TField field = new TField();
 if (this.values != null) {
 field.name = "values";
 field.type = TType.LIST;
-field.id = -1;
+field.id = 1;
 oprot.writeFieldBegin(field);
 {
   oprot.writeListBegin(new TList(TType.STRING, this.values.size()));
@@ -386,38 +386,38 @@ oprot.writeFieldBegin(field);
 }
 oprot.writeFieldEnd();
 }
-if (this.database != null) {
-field.name = "database";
+if (this.dbName != null) {
+field.name = "dbName";
 field.type = TType.STRING;
-field.id = -2;
+field.id = 2;
 oprot.writeFieldBegin(field);
-oprot.writeString(this.database);
+oprot.writeString(this.dbName);
 oprot.writeFieldEnd();
 }
 if (this.tableName != null) {
 field.name = "tableName";
 field.type = TType.STRING;
-field.id = -3;
+field.id = 3;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.tableName);
 oprot.writeFieldEnd();
 }
 field.name = "createTime";
 field.type = TType.I32;
-field.id = -4;
+field.id = 4;
 oprot.writeFieldBegin(field);
 oprot.writeI32(this.createTime);
 oprot.writeFieldEnd();
 field.name = "lastAccessTime";
 field.type = TType.I32;
-field.id = -5;
+field.id = 5;
 oprot.writeFieldBegin(field);
 oprot.writeI32(this.lastAccessTime);
 oprot.writeFieldEnd();
 if (this.sd != null) {
 field.name = "sd";
 field.type = TType.STRUCT;
-field.id = -6;
+field.id = 6;
 oprot.writeFieldBegin(field);
 this.sd.write(oprot);
 oprot.writeFieldEnd();
@@ -425,7 +425,7 @@ oprot.writeFieldEnd();
 if (this.parameters != null) {
 field.name = "parameters";
 field.type = TType.MAP;
-field.id = -7;
+field.id = 7;
 oprot.writeFieldBegin(field);
 {
   oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.parameters.size()));
@@ -445,8 +445,8 @@ public String toString() {
 StringBuilder sb = new StringBuilder("Partition(");
 sb.append("values:");
 sb.append(this.values);
-sb.append(",database:");
-sb.append(this.database);
+sb.append(",dbName:");
+sb.append(this.dbName);
 sb.append(",tableName:");
 sb.append(this.tableName);
 sb.append(",createTime:");

+ 6 - 276
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java

@@ -18,25 +18,13 @@ import com.facebook.thrift.transport.*;
 
 public class SerDeInfo implements TBase, java.io.Serializable {
 private String name;
-private String serializationFormat;
-private String serializationClass;
 private String serializationLib;
-private String fieldDelim;
-private String collectionItemDelim;
-private String mapKeyDelim;
-private String lineDelim;
 private Map<String,String> parameters;
 
 public final Isset __isset = new Isset();
 public static final class Isset implements java.io.Serializable {
 public boolean name = false;
-public boolean serializationFormat = false;
-public boolean serializationClass = false;
 public boolean serializationLib = false;
-public boolean fieldDelim = false;
-public boolean collectionItemDelim = false;
-public boolean mapKeyDelim = false;
-public boolean lineDelim = false;
 public boolean parameters = false;
 }
 
@@ -45,32 +33,14 @@ public SerDeInfo() {
 
 public SerDeInfo(
 String name,
-String serializationFormat,
-String serializationClass,
 String serializationLib,
-String fieldDelim,
-String collectionItemDelim,
-String mapKeyDelim,
-String lineDelim,
 Map<String,String> parameters)
 {
 this();
 this.name = name;
 this.__isset.name = true;
-this.serializationFormat = serializationFormat;
-this.__isset.serializationFormat = true;
-this.serializationClass = serializationClass;
-this.__isset.serializationClass = true;
 this.serializationLib = serializationLib;
 this.__isset.serializationLib = true;
-this.fieldDelim = fieldDelim;
-this.__isset.fieldDelim = true;
-this.collectionItemDelim = collectionItemDelim;
-this.__isset.collectionItemDelim = true;
-this.mapKeyDelim = mapKeyDelim;
-this.__isset.mapKeyDelim = true;
-this.lineDelim = lineDelim;
-this.__isset.lineDelim = true;
 this.parameters = parameters;
 this.__isset.parameters = true;
 }
@@ -88,32 +58,6 @@ public void unsetName() {
 this.__isset.name = false;
 }
 
-public String getSerializationFormat() {
-return this.serializationFormat;
-}
-
-public void setSerializationFormat(String serializationFormat) {
-this.serializationFormat = serializationFormat;
-this.__isset.serializationFormat = true;
-}
-
-public void unsetSerializationFormat() {
-this.__isset.serializationFormat = false;
-}
-
-public String getSerializationClass() {
-return this.serializationClass;
-}
-
-public void setSerializationClass(String serializationClass) {
-this.serializationClass = serializationClass;
-this.__isset.serializationClass = true;
-}
-
-public void unsetSerializationClass() {
-this.__isset.serializationClass = false;
-}
-
 public String getSerializationLib() {
 return this.serializationLib;
 }
@@ -127,58 +71,6 @@ public void unsetSerializationLib() {
 this.__isset.serializationLib = false;
 }
 
-public String getFieldDelim() {
-return this.fieldDelim;
-}
-
-public void setFieldDelim(String fieldDelim) {
-this.fieldDelim = fieldDelim;
-this.__isset.fieldDelim = true;
-}
-
-public void unsetFieldDelim() {
-this.__isset.fieldDelim = false;
-}
-
-public String getCollectionItemDelim() {
-return this.collectionItemDelim;
-}
-
-public void setCollectionItemDelim(String collectionItemDelim) {
-this.collectionItemDelim = collectionItemDelim;
-this.__isset.collectionItemDelim = true;
-}
-
-public void unsetCollectionItemDelim() {
-this.__isset.collectionItemDelim = false;
-}
-
-public String getMapKeyDelim() {
-return this.mapKeyDelim;
-}
-
-public void setMapKeyDelim(String mapKeyDelim) {
-this.mapKeyDelim = mapKeyDelim;
-this.__isset.mapKeyDelim = true;
-}
-
-public void unsetMapKeyDelim() {
-this.__isset.mapKeyDelim = false;
-}
-
-public String getLineDelim() {
-return this.lineDelim;
-}
-
-public void setLineDelim(String lineDelim) {
-this.lineDelim = lineDelim;
-this.__isset.lineDelim = true;
-}
-
-public void unsetLineDelim() {
-this.__isset.lineDelim = false;
-}
-
 public int getParametersSize() {
 return (this.parameters == null) ? 0 : this.parameters.size();
 }
@@ -226,24 +118,6 @@ if (!this.name.equals(that.name))
   return false;
 }
 
-boolean this_present_serializationFormat = true && (this.serializationFormat != null);
-boolean that_present_serializationFormat = true && (that.serializationFormat != null);
-if (this_present_serializationFormat || that_present_serializationFormat) {
-if (!(this_present_serializationFormat && that_present_serializationFormat))
-  return false;
-if (!this.serializationFormat.equals(that.serializationFormat))
-  return false;
-}
-
-boolean this_present_serializationClass = true && (this.serializationClass != null);
-boolean that_present_serializationClass = true && (that.serializationClass != null);
-if (this_present_serializationClass || that_present_serializationClass) {
-if (!(this_present_serializationClass && that_present_serializationClass))
-  return false;
-if (!this.serializationClass.equals(that.serializationClass))
-  return false;
-}
-
 boolean this_present_serializationLib = true && (this.serializationLib != null);
 boolean that_present_serializationLib = true && (that.serializationLib != null);
 if (this_present_serializationLib || that_present_serializationLib) {
@@ -253,42 +127,6 @@ if (!this.serializationLib.equals(that.serializationLib))
   return false;
 }
 
-boolean this_present_fieldDelim = true && (this.fieldDelim != null);
-boolean that_present_fieldDelim = true && (that.fieldDelim != null);
-if (this_present_fieldDelim || that_present_fieldDelim) {
-if (!(this_present_fieldDelim && that_present_fieldDelim))
-  return false;
-if (!this.fieldDelim.equals(that.fieldDelim))
-  return false;
-}
-
-boolean this_present_collectionItemDelim = true && (this.collectionItemDelim != null);
-boolean that_present_collectionItemDelim = true && (that.collectionItemDelim != null);
-if (this_present_collectionItemDelim || that_present_collectionItemDelim) {
-if (!(this_present_collectionItemDelim && that_present_collectionItemDelim))
-  return false;
-if (!this.collectionItemDelim.equals(that.collectionItemDelim))
-  return false;
-}
-
-boolean this_present_mapKeyDelim = true && (this.mapKeyDelim != null);
-boolean that_present_mapKeyDelim = true && (that.mapKeyDelim != null);
-if (this_present_mapKeyDelim || that_present_mapKeyDelim) {
-if (!(this_present_mapKeyDelim && that_present_mapKeyDelim))
-  return false;
-if (!this.mapKeyDelim.equals(that.mapKeyDelim))
-  return false;
-}
-
-boolean this_present_lineDelim = true && (this.lineDelim != null);
-boolean that_present_lineDelim = true && (that.lineDelim != null);
-if (this_present_lineDelim || that_present_lineDelim) {
-if (!(this_present_lineDelim && that_present_lineDelim))
-  return false;
-if (!this.lineDelim.equals(that.lineDelim))
-  return false;
-}
-
 boolean this_present_parameters = true && (this.parameters != null);
 boolean that_present_parameters = true && (that.parameters != null);
 if (this_present_parameters || that_present_parameters) {
@@ -316,7 +154,7 @@ if (field.type == TType.STOP) {
 }
 switch (field.id)
 {
-  case -1:
+  case 1:
     if (field.type == TType.STRING) {
       this.name = iprot.readString();
       this.__isset.name = true;
@@ -324,23 +162,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -2:
-    if (field.type == TType.STRING) {
-      this.serializationFormat = iprot.readString();
-      this.__isset.serializationFormat = true;
-    } else { 
-      TProtocolUtil.skip(iprot, field.type);
-    }
-    break;
-  case -3:
-    if (field.type == TType.STRING) {
-      this.serializationClass = iprot.readString();
-      this.__isset.serializationClass = true;
-    } else { 
-      TProtocolUtil.skip(iprot, field.type);
-    }
-    break;
-  case -4:
+  case 2:
     if (field.type == TType.STRING) {
       this.serializationLib = iprot.readString();
       this.__isset.serializationLib = true;
@@ -348,39 +170,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -5:
-    if (field.type == TType.STRING) {
-      this.fieldDelim = iprot.readString();
-      this.__isset.fieldDelim = true;
-    } else { 
-      TProtocolUtil.skip(iprot, field.type);
-    }
-    break;
-  case -6:
-    if (field.type == TType.STRING) {
-      this.collectionItemDelim = iprot.readString();
-      this.__isset.collectionItemDelim = true;
-    } else { 
-      TProtocolUtil.skip(iprot, field.type);
-    }
-    break;
-  case -7:
-    if (field.type == TType.STRING) {
-      this.mapKeyDelim = iprot.readString();
-      this.__isset.mapKeyDelim = true;
-    } else { 
-      TProtocolUtil.skip(iprot, field.type);
-    }
-    break;
-  case -8:
-    if (field.type == TType.STRING) {
-      this.lineDelim = iprot.readString();
-      this.__isset.lineDelim = true;
-    } else { 
-      TProtocolUtil.skip(iprot, field.type);
-    }
-    break;
-  case -9:
+  case 3:
     if (field.type == TType.MAP) {
       {
         TMap _map4 = iprot.readMapBegin();
@@ -416,71 +206,23 @@ TField field = new TField();
 if (this.name != null) {
 field.name = "name";
 field.type = TType.STRING;
-field.id = -1;
+field.id = 1;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.name);
 oprot.writeFieldEnd();
 }
-if (this.serializationFormat != null) {
-field.name = "serializationFormat";
-field.type = TType.STRING;
-field.id = -2;
-oprot.writeFieldBegin(field);
-oprot.writeString(this.serializationFormat);
-oprot.writeFieldEnd();
-}
-if (this.serializationClass != null) {
-field.name = "serializationClass";
-field.type = TType.STRING;
-field.id = -3;
-oprot.writeFieldBegin(field);
-oprot.writeString(this.serializationClass);
-oprot.writeFieldEnd();
-}
 if (this.serializationLib != null) {
 field.name = "serializationLib";
 field.type = TType.STRING;
-field.id = -4;
+field.id = 2;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.serializationLib);
 oprot.writeFieldEnd();
 }
-if (this.fieldDelim != null) {
-field.name = "fieldDelim";
-field.type = TType.STRING;
-field.id = -5;
-oprot.writeFieldBegin(field);
-oprot.writeString(this.fieldDelim);
-oprot.writeFieldEnd();
-}
-if (this.collectionItemDelim != null) {
-field.name = "collectionItemDelim";
-field.type = TType.STRING;
-field.id = -6;
-oprot.writeFieldBegin(field);
-oprot.writeString(this.collectionItemDelim);
-oprot.writeFieldEnd();
-}
-if (this.mapKeyDelim != null) {
-field.name = "mapKeyDelim";
-field.type = TType.STRING;
-field.id = -7;
-oprot.writeFieldBegin(field);
-oprot.writeString(this.mapKeyDelim);
-oprot.writeFieldEnd();
-}
-if (this.lineDelim != null) {
-field.name = "lineDelim";
-field.type = TType.STRING;
-field.id = -8;
-oprot.writeFieldBegin(field);
-oprot.writeString(this.lineDelim);
-oprot.writeFieldEnd();
-}
 if (this.parameters != null) {
 field.name = "parameters";
 field.type = TType.MAP;
-field.id = -9;
+field.id = 3;
 oprot.writeFieldBegin(field);
 {
   oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.parameters.size()));
@@ -500,20 +242,8 @@ public String toString() {
 StringBuilder sb = new StringBuilder("SerDeInfo(");
 sb.append("name:");
 sb.append(this.name);
-sb.append(",serializationFormat:");
-sb.append(this.serializationFormat);
-sb.append(",serializationClass:");
-sb.append(this.serializationClass);
 sb.append(",serializationLib:");
 sb.append(this.serializationLib);
-sb.append(",fieldDelim:");
-sb.append(this.fieldDelim);
-sb.append(",collectionItemDelim:");
-sb.append(this.collectionItemDelim);
-sb.append(",mapKeyDelim:");
-sb.append(this.mapKeyDelim);
-sb.append(",lineDelim:");
-sb.append(this.lineDelim);
 sb.append(",parameters:");
 sb.append(this.parameters);
 sb.append(")");

+ 43 - 43
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java

@@ -21,7 +21,7 @@ private List<FieldSchema> cols;
 private String location;
 private String inputFormat;
 private String outputFormat;
-private boolean isCompressed;
+private boolean compressed;
 private int numBuckets;
 private SerDeInfo serdeInfo;
 private List<String> bucketCols;
@@ -34,7 +34,7 @@ public boolean cols = false;
 public boolean location = false;
 public boolean inputFormat = false;
 public boolean outputFormat = false;
-public boolean isCompressed = false;
+public boolean compressed = false;
 public boolean numBuckets = false;
 public boolean serdeInfo = false;
 public boolean bucketCols = false;
@@ -52,7 +52,7 @@ List<FieldSchema> cols,
 String location,
 String inputFormat,
 String outputFormat,
-boolean isCompressed,
+boolean compressed,
 int numBuckets,
 SerDeInfo serdeInfo,
 List<String> bucketCols,
@@ -68,8 +68,8 @@ this.inputFormat = inputFormat;
 this.__isset.inputFormat = true;
 this.outputFormat = outputFormat;
 this.__isset.outputFormat = true;
-this.isCompressed = isCompressed;
-this.__isset.isCompressed = true;
+this.compressed = compressed;
+this.__isset.compressed = true;
 this.numBuckets = numBuckets;
 this.__isset.numBuckets = true;
 this.serdeInfo = serdeInfo;
@@ -151,17 +151,17 @@ public void unsetOutputFormat() {
 this.__isset.outputFormat = false;
 }
 
-public boolean isIsCompressed() {
-return this.isCompressed;
+public boolean isCompressed() {
+return this.compressed;
 }
 
-public void setIsCompressed(boolean isCompressed) {
-this.isCompressed = isCompressed;
-this.__isset.isCompressed = true;
+public void setCompressed(boolean compressed) {
+this.compressed = compressed;
+this.__isset.compressed = true;
 }
 
-public void unsetIsCompressed() {
-this.__isset.isCompressed = false;
+public void unsetCompressed() {
+this.__isset.compressed = false;
 }
 
 public int getNumBuckets() {
@@ -325,12 +325,12 @@ if (!this.outputFormat.equals(that.outputFormat))
   return false;
 }
 
-boolean this_present_isCompressed = true;
-boolean that_present_isCompressed = true;
-if (this_present_isCompressed || that_present_isCompressed) {
-if (!(this_present_isCompressed && that_present_isCompressed))
+boolean this_present_compressed = true;
+boolean that_present_compressed = true;
+if (this_present_compressed || that_present_compressed) {
+if (!(this_present_compressed && that_present_compressed))
   return false;
-if (this.isCompressed != that.isCompressed)
+if (this.compressed != that.compressed)
   return false;
 }
 
@@ -397,7 +397,7 @@ if (field.type == TType.STOP) {
 }
 switch (field.id)
 {
-  case -1:
+  case 1:
     if (field.type == TType.LIST) {
       {
         TList _list9 = iprot.readListBegin();
@@ -416,7 +416,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -2:
+  case 2:
     if (field.type == TType.STRING) {
       this.location = iprot.readString();
       this.__isset.location = true;
@@ -424,7 +424,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -3:
+  case 3:
     if (field.type == TType.STRING) {
       this.inputFormat = iprot.readString();
       this.__isset.inputFormat = true;
@@ -432,7 +432,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -4:
+  case 4:
     if (field.type == TType.STRING) {
       this.outputFormat = iprot.readString();
       this.__isset.outputFormat = true;
@@ -440,15 +440,15 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -5:
+  case 5:
     if (field.type == TType.BOOL) {
-      this.isCompressed = iprot.readBool();
-      this.__isset.isCompressed = true;
+      this.compressed = iprot.readBool();
+      this.__isset.compressed = true;
     } else { 
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -6:
+  case 6:
     if (field.type == TType.I32) {
       this.numBuckets = iprot.readI32();
       this.__isset.numBuckets = true;
@@ -456,7 +456,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -7:
+  case 7:
     if (field.type == TType.STRUCT) {
       this.serdeInfo = new SerDeInfo();
       this.serdeInfo.read(iprot);
@@ -465,7 +465,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -8:
+  case 8:
     if (field.type == TType.LIST) {
       {
         TList _list12 = iprot.readListBegin();
@@ -483,7 +483,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -9:
+  case 9:
     if (field.type == TType.LIST) {
       {
         TList _list15 = iprot.readListBegin();
@@ -502,7 +502,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -10:
+  case 10:
     if (field.type == TType.MAP) {
       {
         TMap _map18 = iprot.readMapBegin();
@@ -538,7 +538,7 @@ TField field = new TField();
 if (this.cols != null) {
 field.name = "cols";
 field.type = TType.LIST;
-field.id = -1;
+field.id = 1;
 oprot.writeFieldBegin(field);
 {
   oprot.writeListBegin(new TList(TType.STRUCT, this.cols.size()));
@@ -552,7 +552,7 @@ oprot.writeFieldEnd();
 if (this.location != null) {
 field.name = "location";
 field.type = TType.STRING;
-field.id = -2;
+field.id = 2;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.location);
 oprot.writeFieldEnd();
@@ -560,7 +560,7 @@ oprot.writeFieldEnd();
 if (this.inputFormat != null) {
 field.name = "inputFormat";
 field.type = TType.STRING;
-field.id = -3;
+field.id = 3;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.inputFormat);
 oprot.writeFieldEnd();
@@ -568,27 +568,27 @@ oprot.writeFieldEnd();
 if (this.outputFormat != null) {
 field.name = "outputFormat";
 field.type = TType.STRING;
-field.id = -4;
+field.id = 4;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.outputFormat);
 oprot.writeFieldEnd();
 }
-field.name = "isCompressed";
+field.name = "compressed";
 field.type = TType.BOOL;
-field.id = -5;
+field.id = 5;
 oprot.writeFieldBegin(field);
-oprot.writeBool(this.isCompressed);
+oprot.writeBool(this.compressed);
 oprot.writeFieldEnd();
 field.name = "numBuckets";
 field.type = TType.I32;
-field.id = -6;
+field.id = 6;
 oprot.writeFieldBegin(field);
 oprot.writeI32(this.numBuckets);
 oprot.writeFieldEnd();
 if (this.serdeInfo != null) {
 field.name = "serdeInfo";
 field.type = TType.STRUCT;
-field.id = -7;
+field.id = 7;
 oprot.writeFieldBegin(field);
 this.serdeInfo.write(oprot);
 oprot.writeFieldEnd();
@@ -596,7 +596,7 @@ oprot.writeFieldEnd();
 if (this.bucketCols != null) {
 field.name = "bucketCols";
 field.type = TType.LIST;
-field.id = -8;
+field.id = 8;
 oprot.writeFieldBegin(field);
 {
   oprot.writeListBegin(new TList(TType.STRING, this.bucketCols.size()));
@@ -610,7 +610,7 @@ oprot.writeFieldEnd();
 if (this.sortCols != null) {
 field.name = "sortCols";
 field.type = TType.LIST;
-field.id = -9;
+field.id = 9;
 oprot.writeFieldBegin(field);
 {
   oprot.writeListBegin(new TList(TType.STRUCT, this.sortCols.size()));
@@ -624,7 +624,7 @@ oprot.writeFieldEnd();
 if (this.parameters != null) {
 field.name = "parameters";
 field.type = TType.MAP;
-field.id = -10;
+field.id = 10;
 oprot.writeFieldBegin(field);
 {
   oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.parameters.size()));
@@ -650,8 +650,8 @@ sb.append(",inputFormat:");
 sb.append(this.inputFormat);
 sb.append(",outputFormat:");
 sb.append(this.outputFormat);
-sb.append(",isCompressed:");
-sb.append(this.isCompressed);
+sb.append(",compressed:");
+sb.append(this.compressed);
 sb.append(",numBuckets:");
 sb.append(this.numBuckets);
 sb.append(",serdeInfo:");

+ 42 - 42
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java

@@ -18,7 +18,7 @@ import com.facebook.thrift.transport.*;
 
 public class Table implements TBase, java.io.Serializable {
 private String tableName;
-private String database;
+private String dbName;
 private String owner;
 private int createTime;
 private int lastAccessTime;
@@ -30,7 +30,7 @@ private Map<String,String> parameters;
 public final Isset __isset = new Isset();
 public static final class Isset implements java.io.Serializable {
 public boolean tableName = false;
-public boolean database = false;
+public boolean dbName = false;
 public boolean owner = false;
 public boolean createTime = false;
 public boolean lastAccessTime = false;
@@ -45,7 +45,7 @@ public Table() {
 
 public Table(
 String tableName,
-String database,
+String dbName,
 String owner,
 int createTime,
 int lastAccessTime,
@@ -57,8 +57,8 @@ Map<String,String> parameters)
 this();
 this.tableName = tableName;
 this.__isset.tableName = true;
-this.database = database;
-this.__isset.database = true;
+this.dbName = dbName;
+this.__isset.dbName = true;
 this.owner = owner;
 this.__isset.owner = true;
 this.createTime = createTime;
@@ -88,17 +88,17 @@ public void unsetTableName() {
 this.__isset.tableName = false;
 }
 
-public String getDatabase() {
-return this.database;
+public String getDbName() {
+return this.dbName;
 }
 
-public void setDatabase(String database) {
-this.database = database;
-this.__isset.database = true;
+public void setDbName(String dbName) {
+this.dbName = dbName;
+this.__isset.dbName = true;
 }
 
-public void unsetDatabase() {
-this.__isset.database = false;
+public void unsetDbName() {
+this.__isset.dbName = false;
 }
 
 public String getOwner() {
@@ -244,12 +244,12 @@ if (!this.tableName.equals(that.tableName))
   return false;
 }
 
-boolean this_present_database = true && (this.database != null);
-boolean that_present_database = true && (that.database != null);
-if (this_present_database || that_present_database) {
-if (!(this_present_database && that_present_database))
+boolean this_present_dbName = true && (this.dbName != null);
+boolean that_present_dbName = true && (that.dbName != null);
+if (this_present_dbName || that_present_dbName) {
+if (!(this_present_dbName && that_present_dbName))
   return false;
-if (!this.database.equals(that.database))
+if (!this.dbName.equals(that.dbName))
   return false;
 }
 
@@ -334,7 +334,7 @@ if (field.type == TType.STOP) {
 }
 switch (field.id)
 {
-  case -1:
+  case 1:
     if (field.type == TType.STRING) {
       this.tableName = iprot.readString();
       this.__isset.tableName = true;
@@ -342,15 +342,15 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -2:
+  case 2:
     if (field.type == TType.STRING) {
-      this.database = iprot.readString();
-      this.__isset.database = true;
+      this.dbName = iprot.readString();
+      this.__isset.dbName = true;
     } else { 
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -3:
+  case 3:
     if (field.type == TType.STRING) {
       this.owner = iprot.readString();
       this.__isset.owner = true;
@@ -358,7 +358,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -4:
+  case 4:
     if (field.type == TType.I32) {
       this.createTime = iprot.readI32();
       this.__isset.createTime = true;
@@ -366,7 +366,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -5:
+  case 5:
     if (field.type == TType.I32) {
       this.lastAccessTime = iprot.readI32();
       this.__isset.lastAccessTime = true;
@@ -374,7 +374,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -6:
+  case 6:
     if (field.type == TType.I32) {
       this.retention = iprot.readI32();
       this.__isset.retention = true;
@@ -382,7 +382,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -7:
+  case 7:
     if (field.type == TType.STRUCT) {
       this.sd = new StorageDescriptor();
       this.sd.read(iprot);
@@ -391,7 +391,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -8:
+  case 8:
     if (field.type == TType.LIST) {
       {
         TList _list26 = iprot.readListBegin();
@@ -410,7 +410,7 @@ switch (field.id)
       TProtocolUtil.skip(iprot, field.type);
     }
     break;
-  case -9:
+  case 9:
     if (field.type == TType.MAP) {
       {
         TMap _map29 = iprot.readMapBegin();
@@ -446,49 +446,49 @@ TField field = new TField();
 if (this.tableName != null) {
 field.name = "tableName";
 field.type = TType.STRING;
-field.id = -1;
+field.id = 1;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.tableName);
 oprot.writeFieldEnd();
 }
-if (this.database != null) {
-field.name = "database";
+if (this.dbName != null) {
+field.name = "dbName";
 field.type = TType.STRING;
-field.id = -2;
+field.id = 2;
 oprot.writeFieldBegin(field);
-oprot.writeString(this.database);
+oprot.writeString(this.dbName);
 oprot.writeFieldEnd();
 }
 if (this.owner != null) {
 field.name = "owner";
 field.type = TType.STRING;
-field.id = -3;
+field.id = 3;
 oprot.writeFieldBegin(field);
 oprot.writeString(this.owner);
 oprot.writeFieldEnd();
 }
 field.name = "createTime";
 field.type = TType.I32;
-field.id = -4;
+field.id = 4;
 oprot.writeFieldBegin(field);
 oprot.writeI32(this.createTime);
 oprot.writeFieldEnd();
 field.name = "lastAccessTime";
 field.type = TType.I32;
-field.id = -5;
+field.id = 5;
 oprot.writeFieldBegin(field);
 oprot.writeI32(this.lastAccessTime);
 oprot.writeFieldEnd();
 field.name = "retention";
 field.type = TType.I32;
-field.id = -6;
+field.id = 6;
 oprot.writeFieldBegin(field);
 oprot.writeI32(this.retention);
 oprot.writeFieldEnd();
 if (this.sd != null) {
 field.name = "sd";
 field.type = TType.STRUCT;
-field.id = -7;
+field.id = 7;
 oprot.writeFieldBegin(field);
 this.sd.write(oprot);
 oprot.writeFieldEnd();
@@ -496,7 +496,7 @@ oprot.writeFieldEnd();
 if (this.partitionKeys != null) {
 field.name = "partitionKeys";
 field.type = TType.LIST;
-field.id = -8;
+field.id = 8;
 oprot.writeFieldBegin(field);
 {
   oprot.writeListBegin(new TList(TType.STRUCT, this.partitionKeys.size()));
@@ -510,7 +510,7 @@ oprot.writeFieldEnd();
 if (this.parameters != null) {
 field.name = "parameters";
 field.type = TType.MAP;
-field.id = -9;
+field.id = 9;
 oprot.writeFieldBegin(field);
 {
   oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.parameters.size()));
@@ -530,8 +530,8 @@ public String toString() {
 StringBuilder sb = new StringBuilder("Table(");
 sb.append("tableName:");
 sb.append(this.tableName);
-sb.append(",database:");
-sb.append(this.database);
+sb.append(",dbName:");
+sb.append(this.dbName);
 sb.append(",owner:");
 sb.append(this.owner);
 sb.append(",createTime:");

파일 크기가 너무 크기때문에 변경 상태를 표시하지 않습니다.
+ 123 - 1725
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java


+ 75 - 75
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftMetaStore.java

@@ -19,7 +19,7 @@ import com.facebook.thrift.transport.*;
 public class ThriftMetaStore {
 
 /**
- * This interface is the live.
+ * This interface is deprecated.
  */
 public interface Iface extends com.facebook.fb303.FacebookService.Iface {
 
@@ -1176,14 +1176,14 @@ while (true)
     case 0:
       if (field.type == TType.LIST) {
         {
-          TList _list99 = iprot.readListBegin();
-          this.success = new ArrayList<FieldSchema>(_list99.size);
-          for (int _i100 = 0; _i100 < _list99.size; ++_i100)
+          TList _list89 = iprot.readListBegin();
+          this.success = new ArrayList<FieldSchema>(_list89.size);
+          for (int _i90 = 0; _i90 < _list89.size; ++_i90)
           {
-            FieldSchema _elem101 = new FieldSchema();
-            _elem101 = new FieldSchema();
-            _elem101.read(iprot);
-            this.success.add(_elem101);
+            FieldSchema _elem91 = new FieldSchema();
+            _elem91 = new FieldSchema();
+            _elem91.read(iprot);
+            this.success.add(_elem91);
           }
           iprot.readListEnd();
         }
@@ -1241,8 +1241,8 @@ if (this.__isset.success) {
     oprot.writeFieldBegin(field);
     {
       oprot.writeListBegin(new TList(TType.STRUCT, this.success.size()));
-      for (FieldSchema _iter102 : this.success)      {
-        _iter102.write(oprot);
+      for (FieldSchema _iter92 : this.success)      {
+        _iter92.write(oprot);
       }
       oprot.writeListEnd();
     }
@@ -1630,13 +1630,13 @@ while (true)
     case 0:
       if (field.type == TType.LIST) {
         {
-          TList _list103 = iprot.readListBegin();
-          this.success = new ArrayList<String>(_list103.size);
-          for (int _i104 = 0; _i104 < _list103.size; ++_i104)
+          TList _list93 = iprot.readListBegin();
+          this.success = new ArrayList<String>(_list93.size);
+          for (int _i94 = 0; _i94 < _list93.size; ++_i94)
           {
-            String _elem105 = null;
-            _elem105 = iprot.readString();
-            this.success.add(_elem105);
+            String _elem95 = null;
+            _elem95 = iprot.readString();
+            this.success.add(_elem95);
           }
           iprot.readListEnd();
         }
@@ -1694,8 +1694,8 @@ if (this.__isset.success) {
     oprot.writeFieldBegin(field);
     {
       oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
-      for (String _iter106 : this.success)      {
-        oprot.writeString(_iter106);
+      for (String _iter96 : this.success)      {
+        oprot.writeString(_iter96);
       }
       oprot.writeListEnd();
     }
@@ -2034,15 +2034,15 @@ while (true)
     case 0:
       if (field.type == TType.MAP) {
         {
-          TMap _map107 = iprot.readMapBegin();
-          this.success = new HashMap<String,String>(2*_map107.size);
-          for (int _i108 = 0; _i108 < _map107.size; ++_i108)
+          TMap _map97 = iprot.readMapBegin();
+          this.success = new HashMap<String,String>(2*_map97.size);
+          for (int _i98 = 0; _i98 < _map97.size; ++_i98)
           {
-            String _key109;
-            String _val110;
-            _key109 = iprot.readString();
-            _val110 = iprot.readString();
-            this.success.put(_key109, _val110);
+            String _key99;
+            String _val100;
+            _key99 = iprot.readString();
+            _val100 = iprot.readString();
+            this.success.put(_key99, _val100);
           }
           iprot.readMapEnd();
         }
@@ -2100,9 +2100,9 @@ if (this.__isset.success) {
     oprot.writeFieldBegin(field);
     {
       oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.success.size()));
-      for (String _iter111 : this.success.keySet())      {
-        oprot.writeString(_iter111);
-        oprot.writeString(this.success.get(_iter111));
+      for (String _iter101 : this.success.keySet())      {
+        oprot.writeString(_iter101);
+        oprot.writeString(this.success.get(_iter101));
       }
       oprot.writeMapEnd();
     }
@@ -2313,15 +2313,15 @@ while (true)
     case -3:
       if (field.type == TType.MAP) {
         {
-          TMap _map112 = iprot.readMapBegin();
-          this.schema = new HashMap<String,String>(2*_map112.size);
-          for (int _i113 = 0; _i113 < _map112.size; ++_i113)
+          TMap _map102 = iprot.readMapBegin();
+          this.schema = new HashMap<String,String>(2*_map102.size);
+          for (int _i103 = 0; _i103 < _map102.size; ++_i103)
           {
-            String _key114;
-            String _val115;
-            _key114 = iprot.readString();
-            _val115 = iprot.readString();
-            this.schema.put(_key114, _val115);
+            String _key104;
+            String _val105;
+            _key104 = iprot.readString();
+            _val105 = iprot.readString();
+            this.schema.put(_key104, _val105);
           }
           iprot.readMapEnd();
         }
@@ -2366,9 +2366,9 @@ if (this.schema != null) {
   oprot.writeFieldBegin(field);
   {
     oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.schema.size()));
-    for (String _iter116 : this.schema.keySet())    {
-      oprot.writeString(_iter116);
-      oprot.writeString(this.schema.get(_iter116));
+    for (String _iter106 : this.schema.keySet())    {
+      oprot.writeString(_iter106);
+      oprot.writeString(this.schema.get(_iter106));
     }
     oprot.writeMapEnd();
   }
@@ -2764,15 +2764,15 @@ while (true)
     case -3:
       if (field.type == TType.MAP) {
         {
-          TMap _map117 = iprot.readMapBegin();
-          this.schema = new HashMap<String,String>(2*_map117.size);
-          for (int _i118 = 0; _i118 < _map117.size; ++_i118)
+          TMap _map107 = iprot.readMapBegin();
+          this.schema = new HashMap<String,String>(2*_map107.size);
+          for (int _i108 = 0; _i108 < _map107.size; ++_i108)
           {
-            String _key119;
-            String _val120;
-            _key119 = iprot.readString();
-            _val120 = iprot.readString();
-            this.schema.put(_key119, _val120);
+            String _key109;
+            String _val110;
+            _key109 = iprot.readString();
+            _val110 = iprot.readString();
+            this.schema.put(_key109, _val110);
           }
           iprot.readMapEnd();
         }
@@ -2817,9 +2817,9 @@ if (this.schema != null) {
   oprot.writeFieldBegin(field);
   {
     oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.schema.size()));
-    for (String _iter121 : this.schema.keySet())    {
-      oprot.writeString(_iter121);
-      oprot.writeString(this.schema.get(_iter121));
+    for (String _iter111 : this.schema.keySet())    {
+      oprot.writeString(_iter111);
+      oprot.writeString(this.schema.get(_iter111));
     }
     oprot.writeMapEnd();
   }
@@ -4507,13 +4507,13 @@ while (true)
     case 0:
       if (field.type == TType.LIST) {
         {
-          TList _list122 = iprot.readListBegin();
-          this.success = new ArrayList<String>(_list122.size);
-          for (int _i123 = 0; _i123 < _list122.size; ++_i123)
+          TList _list112 = iprot.readListBegin();
+          this.success = new ArrayList<String>(_list112.size);
+          for (int _i113 = 0; _i113 < _list112.size; ++_i113)
           {
-            String _elem124 = null;
-            _elem124 = iprot.readString();
-            this.success.add(_elem124);
+            String _elem114 = null;
+            _elem114 = iprot.readString();
+            this.success.add(_elem114);
           }
           iprot.readListEnd();
         }
@@ -4571,8 +4571,8 @@ if (this.__isset.success) {
     oprot.writeFieldBegin(field);
     {
       oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
-      for (String _iter125 : this.success)      {
-        oprot.writeString(_iter125);
+      for (String _iter115 : this.success)      {
+        oprot.writeString(_iter115);
       }
       oprot.writeListEnd();
     }
@@ -4803,13 +4803,13 @@ while (true)
     case 0:
       if (field.type == TType.LIST) {
         {
-          TList _list126 = iprot.readListBegin();
-          this.success = new ArrayList<String>(_list126.size);
-          for (int _i127 = 0; _i127 < _list126.size; ++_i127)
+          TList _list116 = iprot.readListBegin();
+          this.success = new ArrayList<String>(_list116.size);
+          for (int _i117 = 0; _i117 < _list116.size; ++_i117)
           {
-            String _elem128 = null;
-            _elem128 = iprot.readString();
-            this.success.add(_elem128);
+            String _elem118 = null;
+            _elem118 = iprot.readString();
+            this.success.add(_elem118);
           }
           iprot.readListEnd();
         }
@@ -4849,8 +4849,8 @@ if (this.__isset.success) {
     oprot.writeFieldBegin(field);
     {
       oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
-      for (String _iter129 : this.success)      {
-        oprot.writeString(_iter129);
+      for (String _iter119 : this.success)      {
+        oprot.writeString(_iter119);
       }
       oprot.writeListEnd();
     }
@@ -5304,13 +5304,13 @@ while (true)
     case 0:
       if (field.type == TType.LIST) {
         {
-          TList _list130 = iprot.readListBegin();
-          this.success = new ArrayList<String>(_list130.size);
-          for (int _i131 = 0; _i131 < _list130.size; ++_i131)
+          TList _list120 = iprot.readListBegin();
+          this.success = new ArrayList<String>(_list120.size);
+          for (int _i121 = 0; _i121 < _list120.size; ++_i121)
           {
-            String _elem132 = null;
-            _elem132 = iprot.readString();
-            this.success.add(_elem132);
+            String _elem122 = null;
+            _elem122 = iprot.readString();
+            this.success.add(_elem122);
           }
           iprot.readListEnd();
         }
@@ -5368,8 +5368,8 @@ if (this.__isset.success) {
     oprot.writeFieldBegin(field);
     {
       oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
-      for (String _iter133 : this.success)      {
-        oprot.writeString(_iter133);
+      for (String _iter123 : this.success)      {
+        oprot.writeString(_iter123);
       }
       oprot.writeListEnd();
     }

+ 8 - 8
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java

@@ -185,7 +185,7 @@ public class Type implements TBase, java.io.Serializable {
       }
       switch (field.id)
       {
-        case -1:
+        case 1:
           if (field.type == TType.STRING) {
             this.name = iprot.readString();
             this.__isset.name = true;
@@ -193,7 +193,7 @@ public class Type implements TBase, java.io.Serializable {
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case -2:
+        case 2:
           if (field.type == TType.STRING) {
             this.type1 = iprot.readString();
             this.__isset.type1 = true;
@@ -201,7 +201,7 @@ public class Type implements TBase, java.io.Serializable {
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case -3:
+        case 3:
           if (field.type == TType.STRING) {
             this.type2 = iprot.readString();
             this.__isset.type2 = true;
@@ -209,7 +209,7 @@ public class Type implements TBase, java.io.Serializable {
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case -4:
+        case 4:
           if (field.type == TType.LIST) {
             {
               TList _list0 = iprot.readListBegin();
@@ -244,7 +244,7 @@ public class Type implements TBase, java.io.Serializable {
     if (this.name != null) {
       field.name = "name";
       field.type = TType.STRING;
-      field.id = -1;
+      field.id = 1;
       oprot.writeFieldBegin(field);
       oprot.writeString(this.name);
       oprot.writeFieldEnd();
@@ -253,7 +253,7 @@ public class Type implements TBase, java.io.Serializable {
       if (this.__isset.type1) {
       field.name = "type1";
       field.type = TType.STRING;
-      field.id = -2;
+      field.id = 2;
       oprot.writeFieldBegin(field);
       oprot.writeString(this.type1);
       oprot.writeFieldEnd();
@@ -263,7 +263,7 @@ public class Type implements TBase, java.io.Serializable {
     if (this.__isset.type2) {
     field.name = "type2";
     field.type = TType.STRING;
-    field.id = -3;
+    field.id = 3;
     oprot.writeFieldBegin(field);
     oprot.writeString(this.type2);
     oprot.writeFieldEnd();
@@ -273,7 +273,7 @@ if (this.fields != null) {
   if (this.__isset.fields) {
   field.name = "fields";
   field.type = TType.LIST;
-  field.id = -4;
+  field.id = 4;
   oprot.writeFieldBegin(field);
   {
     oprot.writeListBegin(new TList(TType.STRUCT, this.fields.size()));

+ 108 - 108
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownDBException.java

@@ -17,116 +17,116 @@ import com.facebook.thrift.protocol.*;
 import com.facebook.thrift.transport.*;
 
 public class UnknownDBException extends Exception implements TBase, java.io.Serializable {
-  private String message;
-
-  public final Isset __isset = new Isset();
-  public static final class Isset implements java.io.Serializable {
-    public boolean message = false;
-  }
-
-  public UnknownDBException() {
-  }
-
-  public UnknownDBException(
-    String message)
-  {
-    this();
-    this.message = message;
-    this.__isset.message = true;
-  }
-
-  public String getMessage() {
-    return this.message;
-  }
-
-  public void setMessage(String message) {
-    this.message = message;
-    this.__isset.message = true;
-  }
-
-  public void unsetMessage() {
-    this.__isset.message = false;
-  }
-
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof UnknownDBException)
-      return this.equals((UnknownDBException)that);
-    return false;
-  }
-
-  public boolean equals(UnknownDBException that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_message = true && (this.message != null);
-    boolean that_present_message = true && (that.message != null);
-    if (this_present_message || that_present_message) {
-      if (!(this_present_message && that_present_message))
-        return false;
-      if (!this.message.equals(that.message))
-        return false;
-    }
+private String message;
 
-    return true;
-  }
-
-  public int hashCode() {
-    return 0;
-  }
-
-  public void read(TProtocol iprot) throws TException {
-    TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == TType.STOP) { 
-        break;
-      }
-      switch (field.id)
-      {
-        case -1:
-          if (field.type == TType.STRING) {
-            this.message = iprot.readString();
-            this.__isset.message = true;
-          } else { 
-            TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          TProtocolUtil.skip(iprot, field.type);
-          break;
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-  }
-
-  public void write(TProtocol oprot) throws TException {
-    TStruct struct = new TStruct("UnknownDBException");
-    oprot.writeStructBegin(struct);
-    TField field = new TField();
-    if (this.message != null) {
-      field.name = "message";
-      field.type = TType.STRING;
-      field.id = -1;
-      oprot.writeFieldBegin(field);
-      oprot.writeString(this.message);
-      oprot.writeFieldEnd();
+public final Isset __isset = new Isset();
+public static final class Isset implements java.io.Serializable {
+public boolean message = false;
+}
+
+public UnknownDBException() {
+}
+
+public UnknownDBException(
+String message)
+{
+this();
+this.message = message;
+this.__isset.message = true;
+}
+
+public String getMessage() {
+return this.message;
+}
+
+public void setMessage(String message) {
+this.message = message;
+this.__isset.message = true;
+}
+
+public void unsetMessage() {
+this.__isset.message = false;
+}
+
+public boolean equals(Object that) {
+if (that == null)
+  return false;
+if (that instanceof UnknownDBException)
+  return this.equals((UnknownDBException)that);
+return false;
+}
+
+public boolean equals(UnknownDBException that) {
+if (that == null)
+  return false;
+
+boolean this_present_message = true && (this.message != null);
+boolean that_present_message = true && (that.message != null);
+if (this_present_message || that_present_message) {
+if (!(this_present_message && that_present_message))
+  return false;
+if (!this.message.equals(that.message))
+  return false;
+}
+
+return true;
+}
+
+public int hashCode() {
+return 0;
+}
+
+public void read(TProtocol iprot) throws TException {
+TField field;
+iprot.readStructBegin();
+while (true)
+{
+field = iprot.readFieldBegin();
+if (field.type == TType.STOP) { 
+  break;
+}
+switch (field.id)
+{
+  case -1:
+    if (field.type == TType.STRING) {
+      this.message = iprot.readString();
+      this.__isset.message = true;
+    } else { 
+      TProtocolUtil.skip(iprot, field.type);
     }
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
-  }
-
-  public String toString() {
-    StringBuilder sb = new StringBuilder("UnknownDBException(");
-    sb.append("message:");
-    sb.append(this.message);
-    sb.append(")");
-    return sb.toString();
-  }
+    break;
+  default:
+    TProtocolUtil.skip(iprot, field.type);
+    break;
+}
+iprot.readFieldEnd();
+}
+iprot.readStructEnd();
+}
+
+public void write(TProtocol oprot) throws TException {
+TStruct struct = new TStruct("UnknownDBException");
+oprot.writeStructBegin(struct);
+TField field = new TField();
+if (this.message != null) {
+field.name = "message";
+field.type = TType.STRING;
+field.id = -1;
+oprot.writeFieldBegin(field);
+oprot.writeString(this.message);
+oprot.writeFieldEnd();
+}
+oprot.writeFieldStop();
+oprot.writeStructEnd();
+}
+
+public String toString() {
+StringBuilder sb = new StringBuilder("UnknownDBException(");
+sb.append("message:");
+sb.append(this.message);
+sb.append(")");
+return sb.toString();
+}
 
 }
 

+ 108 - 108
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/UnknownTableException.java

@@ -17,116 +17,116 @@ import com.facebook.thrift.protocol.*;
 import com.facebook.thrift.transport.*;
 
 public class UnknownTableException extends Exception implements TBase, java.io.Serializable {
-  private String message;
-
-  public final Isset __isset = new Isset();
-  public static final class Isset implements java.io.Serializable {
-    public boolean message = false;
-  }
-
-  public UnknownTableException() {
-  }
-
-  public UnknownTableException(
-    String message)
-  {
-    this();
-    this.message = message;
-    this.__isset.message = true;
-  }
-
-  public String getMessage() {
-    return this.message;
-  }
-
-  public void setMessage(String message) {
-    this.message = message;
-    this.__isset.message = true;
-  }
-
-  public void unsetMessage() {
-    this.__isset.message = false;
-  }
-
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof UnknownTableException)
-      return this.equals((UnknownTableException)that);
-    return false;
-  }
-
-  public boolean equals(UnknownTableException that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_message = true && (this.message != null);
-    boolean that_present_message = true && (that.message != null);
-    if (this_present_message || that_present_message) {
-      if (!(this_present_message && that_present_message))
-        return false;
-      if (!this.message.equals(that.message))
-        return false;
-    }
+private String message;
 
-    return true;
-  }
-
-  public int hashCode() {
-    return 0;
-  }
-
-  public void read(TProtocol iprot) throws TException {
-    TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == TType.STOP) { 
-        break;
-      }
-      switch (field.id)
-      {
-        case -1:
-          if (field.type == TType.STRING) {
-            this.message = iprot.readString();
-            this.__isset.message = true;
-          } else { 
-            TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          TProtocolUtil.skip(iprot, field.type);
-          break;
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-  }
-
-  public void write(TProtocol oprot) throws TException {
-    TStruct struct = new TStruct("UnknownTableException");
-    oprot.writeStructBegin(struct);
-    TField field = new TField();
-    if (this.message != null) {
-      field.name = "message";
-      field.type = TType.STRING;
-      field.id = -1;
-      oprot.writeFieldBegin(field);
-      oprot.writeString(this.message);
-      oprot.writeFieldEnd();
+public final Isset __isset = new Isset();
+public static final class Isset implements java.io.Serializable {
+public boolean message = false;
+}
+
+public UnknownTableException() {
+}
+
+public UnknownTableException(
+String message)
+{
+this();
+this.message = message;
+this.__isset.message = true;
+}
+
+public String getMessage() {
+return this.message;
+}
+
+public void setMessage(String message) {
+this.message = message;
+this.__isset.message = true;
+}
+
+public void unsetMessage() {
+this.__isset.message = false;
+}
+
+public boolean equals(Object that) {
+if (that == null)
+  return false;
+if (that instanceof UnknownTableException)
+  return this.equals((UnknownTableException)that);
+return false;
+}
+
+public boolean equals(UnknownTableException that) {
+if (that == null)
+  return false;
+
+boolean this_present_message = true && (this.message != null);
+boolean that_present_message = true && (that.message != null);
+if (this_present_message || that_present_message) {
+if (!(this_present_message && that_present_message))
+  return false;
+if (!this.message.equals(that.message))
+  return false;
+}
+
+return true;
+}
+
+public int hashCode() {
+return 0;
+}
+
+public void read(TProtocol iprot) throws TException {
+TField field;
+iprot.readStructBegin();
+while (true)
+{
+field = iprot.readFieldBegin();
+if (field.type == TType.STOP) { 
+  break;
+}
+switch (field.id)
+{
+  case -1:
+    if (field.type == TType.STRING) {
+      this.message = iprot.readString();
+      this.__isset.message = true;
+    } else { 
+      TProtocolUtil.skip(iprot, field.type);
     }
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
-  }
-
-  public String toString() {
-    StringBuilder sb = new StringBuilder("UnknownTableException(");
-    sb.append("message:");
-    sb.append(this.message);
-    sb.append(")");
-    return sb.toString();
-  }
+    break;
+  default:
+    TProtocolUtil.skip(iprot, field.type);
+    break;
+}
+iprot.readFieldEnd();
+}
+iprot.readStructEnd();
+}
+
+public void write(TProtocol oprot) throws TException {
+TStruct struct = new TStruct("UnknownTableException");
+oprot.writeStructBegin(struct);
+TField field = new TField();
+if (this.message != null) {
+field.name = "message";
+field.type = TType.STRING;
+field.id = -1;
+oprot.writeFieldBegin(field);
+oprot.writeString(this.message);
+oprot.writeFieldEnd();
+}
+oprot.writeFieldStop();
+oprot.writeStructEnd();
+}
+
+public String toString() {
+StringBuilder sb = new StringBuilder("UnknownTableException(");
+sb.append("message:");
+sb.append(this.message);
+sb.append(")");
+return sb.toString();
+}
 
 }
 

+ 4 - 4
src/contrib/hive/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/Version.java

@@ -114,7 +114,7 @@ public class Version implements TBase, java.io.Serializable {
       }
       switch (field.id)
       {
-        case -1:
+        case 1:
           if (field.type == TType.STRING) {
             this.version = iprot.readString();
             this.__isset.version = true;
@@ -122,7 +122,7 @@ public class Version implements TBase, java.io.Serializable {
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case -2:
+        case 2:
           if (field.type == TType.STRING) {
             this.comments = iprot.readString();
             this.__isset.comments = true;
@@ -146,7 +146,7 @@ public class Version implements TBase, java.io.Serializable {
     if (this.version != null) {
       field.name = "version";
       field.type = TType.STRING;
-      field.id = -1;
+      field.id = 1;
       oprot.writeFieldBegin(field);
       oprot.writeString(this.version);
       oprot.writeFieldEnd();
@@ -154,7 +154,7 @@ public class Version implements TBase, java.io.Serializable {
     if (this.comments != null) {
       field.name = "comments";
       field.type = TType.STRING;
-      field.id = -2;
+      field.id = 2;
       oprot.writeFieldBegin(field);
       oprot.writeString(this.comments);
       oprot.writeFieldEnd();

파일 크기가 너무 크기때문에 변경 상태를 표시하지 않습니다.
+ 100 - 288
src/contrib/hive/metastore/src/gen-php/ThriftHiveMetastore.php


파일 크기가 너무 크기때문에 변경 상태를 표시하지 않습니다.
+ 153 - 153
src/contrib/hive/metastore/src/gen-php/ThriftMetaStore.php


+ 0 - 51
src/contrib/hive/metastore/src/gen-php/hive_metastore_constants.php

@@ -8,8 +8,6 @@ include_once $GLOBALS['THRIFT_ROOT'].'/packages/hive_metastore/hive_metastore_ty
 
 $GLOBALS['hive_metastore_CONSTANTS'] = array();
 
-$GLOBALS['hive_metastore_CONSTANTS']['META_SERDE'] = 'com.facebook.serde.simple_meta.MetadataTypedColumnsetSerDe';
-
 $GLOBALS['hive_metastore_CONSTANTS']['META_TABLE_COLUMNS'] = 'columns';
 
 $GLOBALS['hive_metastore_CONSTANTS']['BUCKET_FIELD_NAME'] = 'bucket_field_name';
@@ -26,59 +24,10 @@ $GLOBALS['hive_metastore_CONSTANTS']['META_TABLE_LOCATION'] = 'location';
 
 $GLOBALS['hive_metastore_CONSTANTS']['META_TABLE_SERDE'] = 'serde';
 
-$GLOBALS['hive_metastore_CONSTANTS']['SERIALIZATION_FORMAT'] = 'serialization.format';
-
-$GLOBALS['hive_metastore_CONSTANTS']['SERIALIZATION_CLASS'] = 'serialization.class';
-
-$GLOBALS['hive_metastore_CONSTANTS']['SERIALIZATION_LIB'] = 'serialization.lib';
-
 $GLOBALS['hive_metastore_CONSTANTS']['META_TABLE_PARTITION_COLUMNS'] = 'partition_columns';
 
 $GLOBALS['hive_metastore_CONSTANTS']['FILE_INPUT_FORMAT'] = 'file.inputformat';
 
 $GLOBALS['hive_metastore_CONSTANTS']['FILE_OUTPUT_FORMAT'] = 'file.outputformat';
 
-$GLOBALS['hive_metastore_CONSTANTS']['KEY_COMMENTS'] = 'key_comments';
-
-$GLOBALS['hive_metastore_CONSTANTS']['VERSION_0_1'] = '0.1';
-
-$GLOBALS['hive_metastore_CONSTANTS']['TINYINT_TYPE_NAME'] = 'tinyint';
-
-$GLOBALS['hive_metastore_CONSTANTS']['INT_TYPE_NAME'] = 'int';
-
-$GLOBALS['hive_metastore_CONSTANTS']['BIGINT_TYPE_NAME'] = 'bigint';
-
-$GLOBALS['hive_metastore_CONSTANTS']['FLOAT_TYPE_NAME'] = 'float';
-
-$GLOBALS['hive_metastore_CONSTANTS']['DOUBLE_TYPE_NAME'] = 'double';
-
-$GLOBALS['hive_metastore_CONSTANTS']['STRING_TYPE_NAME'] = 'string';
-
-$GLOBALS['hive_metastore_CONSTANTS']['DATE_TYPE_NAME'] = 'date';
-
-$GLOBALS['hive_metastore_CONSTANTS']['DATETIME_TYPE_NAME'] = 'datetime';
-
-$GLOBALS['hive_metastore_CONSTANTS']['TIMESTAMP_TYPE_NAME'] = 'timestamp';
-
-$GLOBALS['hive_metastore_CONSTANTS']['LIST_TYPE_NAME'] = 'list';
-
-$GLOBALS['hive_metastore_CONSTANTS']['MAP_TYPE_NAME'] = 'map';
-
-$GLOBALS['hive_metastore_CONSTANTS']['PrimitiveTypes'] = array(
-  'tinyint' => true,
-  'int' => true,
-  'bigint' => true,
-  'float' => true,
-  'double' => true,
-  'string' => true,
-  'date' => true,
-  'datetime' => true,
-  'timestamp' => true,
-);
-
-$GLOBALS['hive_metastore_CONSTANTS']['CollectionTypes'] = array(
-  'list' => true,
-  'map' => true,
-);
-
 ?>

파일 크기가 너무 크기때문에 변경 상태를 표시하지 않습니다.
+ 96 - 432
src/contrib/hive/metastore/src/gen-php/hive_metastore_types.php


+ 6 - 27
src/contrib/hive/metastore/src/gen-py/hive_metastore/ThriftHiveMetastore-remote

@@ -21,7 +21,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print 'Usage: ' + sys.argv[0] + ' [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]'
   print ''
   print 'Functions:'
-  print '  bool create_database(string name, string location_uri)'
+  print '  bool create_database(string name, string description)'
   print '  Database get_database(string name)'
   print '  bool drop_database(string name)'
   print '   get_databases()'
@@ -34,16 +34,13 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print '  void drop_table(string dbname, string name, bool deleteData)'
   print '   get_tables(string db_name, string pattern)'
   print '  Table get_table(string dbname, string tbl_name)'
-  print '  bool set_table_parameters(string dbname, string tbl_name,  params)'
   print '  void alter_table(string dbname, string tbl_name, Table new_tbl)'
-  print '  void truncate_table(string db_name, string table_name, string partition)'
-  print '   cat(string db_name, string table_name, string partition, i32 high)'
   print '  Partition add_partition(Partition new_part)'
   print '  Partition append_partition(string db_name, string tbl_name,  part_vals)'
   print '  bool drop_partition(string db_name, string tbl_name,  part_vals, bool deleteData)'
   print '  Partition get_partition(string db_name, string tbl_name,  part_vals)'
   print '   get_partitions(string db_name, string tbl_name, i16 max_parts)'
-  print '  bool set_partition_parameters(string db_name, string tbl_name, string pname,  params)'
+  print '   get_partition_names(string db_name, string tbl_name, i16 max_parts)'
   print '  bool alter_partitions(StorageDescriptor sd,  parts)'
   print '  bool create_index(Index index_def)'
   print ''
@@ -172,30 +169,12 @@ elif cmd == 'get_table':
     sys.exit(1)
   pp.pprint(client.get_table(args[0],args[1],))
 
-elif cmd == 'set_table_parameters':
-  if len(args) != 3:
-    print 'set_table_parameters requires 3 args'
-    sys.exit(1)
-  pp.pprint(client.set_table_parameters(args[0],args[1],eval(args[2]),))
-
 elif cmd == 'alter_table':
   if len(args) != 3:
     print 'alter_table requires 3 args'
     sys.exit(1)
   pp.pprint(client.alter_table(args[0],args[1],eval(args[2]),))
 
-elif cmd == 'truncate_table':
-  if len(args) != 3:
-    print 'truncate_table requires 3 args'
-    sys.exit(1)
-  pp.pprint(client.truncate_table(args[0],args[1],args[2],))
-
-elif cmd == 'cat':
-  if len(args) != 4:
-    print 'cat requires 4 args'
-    sys.exit(1)
-  pp.pprint(client.cat(args[0],args[1],args[2],eval(args[3]),))
-
 elif cmd == 'add_partition':
   if len(args) != 1:
     print 'add_partition requires 1 args'
@@ -226,11 +205,11 @@ elif cmd == 'get_partitions':
     sys.exit(1)
   pp.pprint(client.get_partitions(args[0],args[1],eval(args[2]),))
 
-elif cmd == 'set_partition_parameters':
-  if len(args) != 4:
-    print 'set_partition_parameters requires 4 args'
+elif cmd == 'get_partition_names':
+  if len(args) != 3:
+    print 'get_partition_names requires 3 args'
     sys.exit(1)
-  pp.pprint(client.set_partition_parameters(args[0],args[1],args[2],eval(args[3]),))
+  pp.pprint(client.get_partition_names(args[0],args[1],eval(args[2]),))
 
 elif cmd == 'alter_partitions':
   if len(args) != 2:

파일 크기가 너무 크기때문에 변경 상태를 표시하지 않습니다.
+ 104 - 801
src/contrib/hive/metastore/src/gen-py/hive_metastore/ThriftHiveMetastore.py


+ 55 - 55
src/contrib/hive/metastore/src/gen-py/hive_metastore/ThriftMetaStore.py

@@ -722,11 +722,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype178, _size175) = iprot.readListBegin()
-          for _i179 in xrange(_size175):
-            _elem180 = FieldSchema()
-            _elem180.read(iprot)
-            self.success.append(_elem180)
+          (_etype160, _size157) = iprot.readListBegin()
+          for _i161 in xrange(_size157):
+            _elem162 = FieldSchema()
+            _elem162.read(iprot)
+            self.success.append(_elem162)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -761,8 +761,8 @@ class get_fields_result:
     if self.success != None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter181 in self.success:
-        iter181.write(oprot)
+      for iter163 in self.success:
+        iter163.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ouch1 != None:
@@ -886,10 +886,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype185, _size182) = iprot.readListBegin()
-          for _i186 in xrange(_size182):
-            _elem187 = iprot.readString();
-            self.success.append(_elem187)
+          (_etype167, _size164) = iprot.readListBegin()
+          for _i168 in xrange(_size164):
+            _elem169 = iprot.readString();
+            self.success.append(_elem169)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -924,8 +924,8 @@ class get_tables_result:
     if self.success != None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter188 in self.success:
-        oprot.writeString(iter188)
+      for iter170 in self.success:
+        oprot.writeString(iter170)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ouch1 != None:
@@ -1037,11 +1037,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype190, _vtype191, _size189 ) = iprot.readMapBegin() 
-          for _i193 in xrange(_size189):
-            _key194 = iprot.readString();
-            _val195 = iprot.readString();
-            self.success[_key194] = _val195
+          (_ktype172, _vtype173, _size171 ) = iprot.readMapBegin() 
+          for _i175 in xrange(_size171):
+            _key176 = iprot.readString();
+            _val177 = iprot.readString();
+            self.success[_key176] = _val177
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -1076,9 +1076,9 @@ class get_schema_result:
     if self.success != None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter196,viter197 in self.success.items():
-        oprot.writeString(kiter196)
-        oprot.writeString(viter197)
+      for kiter178,viter179 in self.success.items():
+        oprot.writeString(kiter178)
+        oprot.writeString(viter179)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.ouch1 != None:
@@ -1145,11 +1145,11 @@ class alter_table_args:
       elif fid == -3:
         if ftype == TType.MAP:
           self.schema = {}
-          (_ktype199, _vtype200, _size198 ) = iprot.readMapBegin() 
-          for _i202 in xrange(_size198):
-            _key203 = iprot.readString();
-            _val204 = iprot.readString();
-            self.schema[_key203] = _val204
+          (_ktype181, _vtype182, _size180 ) = iprot.readMapBegin() 
+          for _i184 in xrange(_size180):
+            _key185 = iprot.readString();
+            _val186 = iprot.readString();
+            self.schema[_key185] = _val186
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -1174,9 +1174,9 @@ class alter_table_args:
     if self.schema != None:
       oprot.writeFieldBegin('schema', TType.MAP, -3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.schema))
-      for kiter205,viter206 in self.schema.items():
-        oprot.writeString(kiter205)
-        oprot.writeString(viter206)
+      for kiter187,viter188 in self.schema.items():
+        oprot.writeString(kiter187)
+        oprot.writeString(viter188)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -1310,11 +1310,11 @@ class create_table_args:
       elif fid == -3:
         if ftype == TType.MAP:
           self.schema = {}
-          (_ktype208, _vtype209, _size207 ) = iprot.readMapBegin() 
-          for _i211 in xrange(_size207):
-            _key212 = iprot.readString();
-            _val213 = iprot.readString();
-            self.schema[_key212] = _val213
+          (_ktype190, _vtype191, _size189 ) = iprot.readMapBegin() 
+          for _i193 in xrange(_size189):
+            _key194 = iprot.readString();
+            _val195 = iprot.readString();
+            self.schema[_key194] = _val195
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -1339,9 +1339,9 @@ class create_table_args:
     if self.schema != None:
       oprot.writeFieldBegin('schema', TType.MAP, -3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.schema))
-      for kiter214,viter215 in self.schema.items():
-        oprot.writeString(kiter214)
-        oprot.writeString(viter215)
+      for kiter196,viter197 in self.schema.items():
+        oprot.writeString(kiter196)
+        oprot.writeString(viter197)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -1959,10 +1959,10 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype219, _size216) = iprot.readListBegin()
-          for _i220 in xrange(_size216):
-            _elem221 = iprot.readString();
-            self.success.append(_elem221)
+          (_etype201, _size198) = iprot.readListBegin()
+          for _i202 in xrange(_size198):
+            _elem203 = iprot.readString();
+            self.success.append(_elem203)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -1997,8 +1997,8 @@ class get_partitions_result:
     if self.success != None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter222 in self.success:
-        oprot.writeString(iter222)
+      for iter204 in self.success:
+        oprot.writeString(iter204)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ouch1 != None:
@@ -2094,10 +2094,10 @@ class get_dbs_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype226, _size223) = iprot.readListBegin()
-          for _i227 in xrange(_size223):
-            _elem228 = iprot.readString();
-            self.success.append(_elem228)
+          (_etype208, _size205) = iprot.readListBegin()
+          for _i209 in xrange(_size205):
+            _elem210 = iprot.readString();
+            self.success.append(_elem210)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -2120,8 +2120,8 @@ class get_dbs_result:
     if self.success != None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter229 in self.success:
-        oprot.writeString(iter229)
+      for iter211 in self.success:
+        oprot.writeString(iter211)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ouch != None:
@@ -2261,10 +2261,10 @@ class cat_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype233, _size230) = iprot.readListBegin()
-          for _i234 in xrange(_size230):
-            _elem235 = iprot.readString();
-            self.success.append(_elem235)
+          (_etype215, _size212) = iprot.readListBegin()
+          for _i216 in xrange(_size212):
+            _elem217 = iprot.readString();
+            self.success.append(_elem217)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -2299,8 +2299,8 @@ class cat_result:
     if self.success != None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter236 in self.success:
-        oprot.writeString(iter236)
+      for iter218 in self.success:
+        oprot.writeString(iter218)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ouch1 != None:

+ 0 - 51
src/contrib/hive/metastore/src/gen-py/hive_metastore/constants.py

@@ -7,8 +7,6 @@
 from thrift.Thrift import *
 from ttypes import *
 
-META_SERDE = 'com.facebook.serde.simple_meta.MetadataTypedColumnsetSerDe'
-
 META_TABLE_COLUMNS = 'columns'
 
 BUCKET_FIELD_NAME = 'bucket_field_name'
@@ -25,58 +23,9 @@ META_TABLE_LOCATION = 'location'
 
 META_TABLE_SERDE = 'serde'
 
-SERIALIZATION_FORMAT = 'serialization.format'
-
-SERIALIZATION_CLASS = 'serialization.class'
-
-SERIALIZATION_LIB = 'serialization.lib'
-
 META_TABLE_PARTITION_COLUMNS = 'partition_columns'
 
 FILE_INPUT_FORMAT = 'file.inputformat'
 
 FILE_OUTPUT_FORMAT = 'file.outputformat'
 
-KEY_COMMENTS = 'key_comments'
-
-VERSION_0_1 = '0.1'
-
-TINYINT_TYPE_NAME = 'tinyint'
-
-INT_TYPE_NAME = 'int'
-
-BIGINT_TYPE_NAME = 'bigint'
-
-FLOAT_TYPE_NAME = 'float'
-
-DOUBLE_TYPE_NAME = 'double'
-
-STRING_TYPE_NAME = 'string'
-
-DATE_TYPE_NAME = 'date'
-
-DATETIME_TYPE_NAME = 'datetime'
-
-TIMESTAMP_TYPE_NAME = 'timestamp'
-
-LIST_TYPE_NAME = 'list'
-
-MAP_TYPE_NAME = 'map'
-
-PrimitiveTypes = set([
-  'tinyint',
-  'int',
-  'bigint',
-  'float',
-  'double',
-  'string',
-  'date',
-  'datetime',
-  'timestamp',
-])
-
-CollectionTypes = set([
-  'list',
-  'map',
-])
-

파일 크기가 너무 크기때문에 변경 상태를 표시하지 않습니다.
+ 186 - 345
src/contrib/hive/metastore/src/gen-py/hive_metastore/ttypes.py


+ 9 - 2
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/FileStore.java

@@ -303,7 +303,7 @@ public class FileStore implements RawStore {
   public void createTable(Table tbl) throws InvalidObjectException, MetaException {
     Properties p = MetaStoreUtils.getSchema(tbl);
     try {
-      DB db = new DB(tbl.getDatabase(), conf);
+      DB db = new DB(tbl.getDbName(), conf);
       RWTable.create(db, tbl.getTableName(), p, conf);
     } catch (UnknownDBException e) {
       throw new InvalidObjectException(e.getMessage());
@@ -325,7 +325,7 @@ public class FileStore implements RawStore {
   public Table getTable(String dbName, String tableName) throws MetaException {
     try {
       Properties p = new DB(dbName, conf).getTable(tableName, true).getSchema();
-      return MetaStoreUtils.getTable(p);
+      return MetaStoreUtils.getTable(conf, p);
     } catch (Exception e) {
       LOG.error(StringUtils.stringifyException(e));
       throw new MetaException(e.getMessage());
@@ -392,4 +392,11 @@ public class FileStore implements RawStore {
     // no-op
   }
 
+  @Override
+  public List<String> listPartitionNames(String db_name, String tbl_name, short max_parts)
+      throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
 };

+ 55 - 125
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java

@@ -19,11 +19,8 @@
 package org.apache.hadoop.hive.metastore;
 
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -46,9 +43,8 @@ import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
-import org.apache.hadoop.hive.serde.SerDe;
-import org.apache.hadoop.hive.serde.SerDeException;
-import org.apache.hadoop.hive.serde.SerDeField;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 
@@ -64,7 +60,7 @@ import com.facebook.thrift.transport.TServerTransport;
 import com.facebook.thrift.transport.TTransportFactory;
 
 /**
- * TODO:pc remove application logic to a separate interface. rename to MetaStoreServer
+ * TODO:pc remove application logic to a separate interface. 
  */
 public class HiveMetaStore extends ThriftHiveMetastore {
   
@@ -271,7 +267,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
       public void create_table(Table tbl) throws AlreadyExistsException, MetaException, InvalidObjectException {
         this.incrementCounter("create_table");
-        logStartFunction("create_table: db=" + tbl.getDatabase() + " tbl=" + tbl.getTableName());
+        logStartFunction("create_table: db=" + tbl.getDbName() + " tbl=" + tbl.getTableName());
         boolean success = false;
         if(!MetaStoreUtils.validateName(tbl.getTableName())) {
           throw new InvalidObjectException(tbl.getTableName() + " is not a valid object name");
@@ -280,17 +276,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           getMS().openTransaction();
           Path tblPath = null;
           if(tbl.getSd().getLocation() == null || tbl.getSd().getLocation().isEmpty()) {
-            tblPath = wh.getDefaultTablePath(tbl.getDatabase(), tbl.getTableName());
+            tblPath = wh.getDefaultTablePath(tbl.getDbName(), tbl.getTableName());
             tbl.getSd().setLocation(tblPath.toString());
           } else {
             tblPath = new Path(tbl.getSd().getLocation());
           }
           // get_table checks whether database exists, it should be moved here
-          try {
-            if(get_table(tbl.getDatabase(), tbl.getTableName()) != null) {
-              throw new AlreadyExistsException("Table " + tbl.getTableName() + " already exists");
-            }
-          } catch (NoSuchObjectException e) {
+          if(is_table_exists(tbl.getDbName(), tbl.getTableName())) {
+            throw new AlreadyExistsException("Table " + tbl.getTableName() + " already exists");
           }
           getMS().createTable(tbl);
           if(wh.mkdirs(tblPath)) {
@@ -304,6 +297,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
       }
       
+      public boolean is_table_exists(String dbname, String name) throws MetaException {
+        try {
+          return (get_table(dbname, name) != null);
+        } catch (NoSuchObjectException e) {
+          return false;
+        }
+      }
+      
       public void drop_table(String dbname, String name, boolean deleteData) throws NoSuchObjectException, MetaException {
         this.incrementCounter("drop_table");
         logStartFunction("drop_table", dbname, name);
@@ -362,16 +363,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         try {
           getMS().openTransaction();
           part = new Partition();
-          part.setDatabase(dbName);
+          part.setDbName(dbName);
           part.setTableName(tableName);
           part.setValues(part_vals);
 
-          Partition old_part = this.get_partition(part.getDatabase(), part.getTableName(), part.getValues());
+          Partition old_part = this.get_partition(part.getDbName(), part.getTableName(), part.getValues());
           if( old_part != null) {
             throw new AlreadyExistsException("Partition already exists:" + part);
           }
           
-          Table tbl = getMS().getTable(part.getDatabase(), part.getTableName());
+          Table tbl = getMS().getTable(part.getDbName(), part.getTableName());
           if(tbl == null) {
             throw new InvalidObjectException("Unable to add partition because table or database do not exist");
           }
@@ -393,19 +394,43 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
         return part;
       }
+      
+      public int add_partitions(List<Partition> parts) throws MetaException, InvalidObjectException, AlreadyExistsException {
+        this.incrementCounter("add_partition");
+        if(parts.size() == 0) {
+          return 0;
+        }
+        String db = parts.get(0).getDbName();
+        String tbl = parts.get(0).getTableName();
+        logStartFunction("add_partitions", db, tbl);
+        boolean success = false;
+        try {
+          getMS().openTransaction();
+          for (Partition part : parts) {
+            this.add_partition(part);
+          }
+          success = true;
+          getMS().commitTransaction();
+        } finally {
+          if(!success) {
+            getMS().rollbackTransaction();
+          }
+        }
+        return parts.size();
+      }
 
       public Partition add_partition(Partition part) throws InvalidObjectException,
           AlreadyExistsException, MetaException {
         this.incrementCounter("add_partition");
-        logStartFunction("add_partition", part.getDatabase(), part.getTableName());
+        logStartFunction("add_partition", part.getDbName(), part.getTableName());
         boolean success = false;
         try {
           getMS().openTransaction();
-          Partition old_part = this.get_partition(part.getDatabase(), part.getTableName(), part.getValues());
+          Partition old_part = this.get_partition(part.getDbName(), part.getTableName(), part.getValues());
           if( old_part != null) {
             throw new AlreadyExistsException("Partition already exists:" + part);
           }
-          Table tbl = getMS().getTable(part.getDatabase(), part.getTableName());
+          Table tbl = getMS().getTable(part.getDbName(), part.getTableName());
           if(tbl == null) {
             throw new InvalidObjectException("Unable to add partition because table or database do not exist");
           }
@@ -446,6 +471,12 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         logStartFunction("get_partitions", db_name, tbl_name);
         return getMS().getPartitions(db_name, tbl_name, max_parts);
       }
+      
+      public List<String> get_partition_names(String db_name, String tbl_name, short max_parts) throws MetaException {
+        this.incrementCounter("get_partition_names");
+        logStartFunction("get_partition_names", db_name, tbl_name);
+        return getMS().listPartitionNames(db_name, tbl_name, max_parts);
+      }
 
       public boolean alter_partitions(StorageDescriptor sd, List<String> parts) throws InvalidOperationException,
           MetaException {
@@ -490,15 +521,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
       }
 
-      @Override
-      public List<String> cat(String db_name, String table_name, String partition, int high)
-          throws MetaException, UnknownDBException, UnknownTableException {
-        this.incrementCounter("cat");
-        logStartFunction("cat: db=" + db_name + " tbl=" + table_name + " part=" + partition + " high=" + high);
-        // TODO Auto-generated method stub
-        throw new MetaException("Not implemented. Please use select * query instead");
-      }
-
       @Override
       public List<String> get_tables(String dbname, String pattern) throws MetaException {
         this.incrementCounter("get_tables");
@@ -506,75 +528,13 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         return getMS().getTables(dbname, pattern);
       }
 
-      @Override
-      public void truncate_table(String db_name, String table_name, String partition)
-          throws MetaException, UnknownTableException, UnknownDBException {
-        // TODO Auto-generated method stub
-        this.incrementCounter("truncate_table");
-        logStartFunction("truncate_table: db=" + db_name + " tbl=" + table_name);
-      }
-      
-      /**
-       * normalizeType
-       *
-       * For pretty printing
-       *
-       * @param type a type name
-       * @return cleaned up - remove Java.lang and make numbers into int , ...
-       */
-      public String normalizeType(String type) {
-        Pattern tpat = Pattern.compile("java.lang.");
-        Matcher m = tpat.matcher(type);
-        type = m.replaceFirst("");
-        String ret = type;
-        if(type.equals("String")) {
-          ret = "string";
-        }  else if(type.equals("Integer")) {
-          ret = "int";
-        }
-        return ret;
-
-      }
 
-      /**
-       * hfToStrting
-       *
-       * Converts a basic SerDe's type to a string. It doesn't do any recursion.
-       *
-       * @param f the hive field - cannot be null!
-       * @return a string representation of the field's type
-       * @exception MetaException - if the SerDe raises it or the field is null
-       *
-       */
-      private String hfToString(SerDeField f) throws MetaException {
-        String ret;
-
-        try {
-          if(f.isPrimitive()) {
-            ret = this.normalizeType(f.getType().getName());
-          } else if(f.isList()) {
-            ret = "List<" +  this.normalizeType(f.getListElementType().getName()) + ">";
-          } else if(f.isMap()) {
-            ret = "Map<" + this.normalizeType(f.getMapKeyType().getName()) + "," +
-              this.normalizeType(f.getMapValueType().getName()) + ">";
-          } else {
-            // complex type and we just show the type name of the complex type
-            ret = f.getName();
-          }
-        }  catch(Exception e) {
-          StringUtils.stringifyException(e);
-          throw new MetaException(e.getMessage());
-        }
-        return ret;
-      }
-
-      public ArrayList<FieldSchema> get_fields(String db, String tableName) throws MetaException,UnknownTableException, UnknownDBException {
+      public List<FieldSchema> get_fields(String db, String tableName) 
+        throws MetaException,UnknownTableException, UnknownDBException {
         this.incrementCounter("get_fields");
         logStartFunction("get_fields: db=" + db + "tbl=" + tableName);
-        ArrayList<FieldSchema> str_fields = new ArrayList<FieldSchema>();
         String [] names = tableName.split("\\.");
         String base_table_name = names[0];
-        List<SerDeField> hive_fields = new ArrayList<SerDeField>();
 
         Table tbl;
         try {
@@ -582,39 +542,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         } catch (NoSuchObjectException e) {
           throw new UnknownTableException(e.getMessage());
         }
-        SerDeField hf = null;
         try {
-          // TODO:pc getSerDe requires 'columns' field in some cases and this doesn't supply
-          SerDe s = MetaStoreUtils.getSerDe(this.hiveConf, tbl);
-
-          // recurse down the type.subtype.subsubtype expression until at the desired type
-          for(int i =  1; i < names.length; i++) {
-            hf = s.getFieldFromExpression(hf,names[i]);
-          }
-
-          // rules on how to recurse the SerDe based on its type
-          if(hf != null && hf.isPrimitive()) {
-            hive_fields.add(hf);
-          } else if(hf != null && hf.isList()) {
-            // don't remember why added this rule??
-            // should just be a hive_fields.add(hf)
-            try {
-              hive_fields = s.getFields(hf);
-            } catch(Exception e) {
-              hive_fields.add(hf);
-            }
-          } else if(hf != null && hf.isMap()) {
-            hive_fields.add(hf);
-          } else {
-            hive_fields = s.getFields(hf);
-          }
-
-          for(SerDeField field: hive_fields) {
-            String name = field.getName();
-            String schema = this.hfToString(field);
-            str_fields.add(new FieldSchema(name, schema, "automatically generated"));
-          }
-          return str_fields;
+          Deserializer s = MetaStoreUtils.getDeserializer(this.hiveConf, tbl);
+          return MetaStoreUtils.getFieldsFromDeserializer(tableName, s);
         } catch(SerDeException e) {
           StringUtils.stringifyException(e);
           throw new MetaException(e.getMessage());

+ 8 - 73
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java

@@ -20,31 +20,24 @@ package org.apache.hadoop.hive.metastore;
 
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.Constants;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.ExistingDependentsException;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
 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.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
-import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 
 import com.facebook.thrift.TException;
 import com.facebook.thrift.protocol.TBinaryProtocol;
@@ -54,8 +47,7 @@ import com.facebook.thrift.transport.TTransport;
 import com.facebook.thrift.transport.TTransportException;
 
 /**
- * Metastore Client.
- * TODO: rename this as MetaStoreClient and remove the interface as it is not needed 
+ * Hive Metastore Client.
  */
 public class HiveMetaStoreClient implements IMetaStoreClient {
   ThriftHiveMetastore.Iface client = null;
@@ -451,70 +443,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     return client.get_type(name);
   }
 
-  @SuppressWarnings("unused")
-  // Will be removed after testing
-  private void createTable(String tableName, Properties schema) throws MetaException,
-      UnknownTableException, TException {
-    Table t = new Table();
-    t.setSd(new StorageDescriptor());
-    t.setTableName(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME));
-    t.getSd().setLocation(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_LOCATION));
-    t.getSd().setInputFormat(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_INPUT_FORMAT,
-          org.apache.hadoop.mapred.SequenceFileInputFormat.class.getName())); 
-    t.getSd().setOutputFormat(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT,
-          org.apache.hadoop.mapred.SequenceFileOutputFormat.class.getName())); 
-    t.setPartitionKeys(new ArrayList<FieldSchema>());
-    t.setDatabase(MetaStoreUtils.DEFAULT_DATABASE_NAME);
-    String part_cols_str = schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS);
-    if (part_cols_str != null && (part_cols_str.trim().length() != 0)) {
-      String [] part_keys = part_cols_str.trim().split("/");
-      for (String key: part_keys) {
-        FieldSchema part = new FieldSchema();
-        part.setName(key);
-        part.setType(Constants.STRING_TYPE_NAME); // default partition key
-        t.getPartitionKeys().add(part);
-      }
-    }
-    t.getSd().setNumBuckets(Integer.parseInt(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_COUNT, "-1")));
-    String bucketFieldName = schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_FIELD_NAME);
-    if ((bucketFieldName != null) && (bucketFieldName.trim().length() != 0)) {
-      t.getSd().setBucketCols(new ArrayList<String>(1));
-      t.getSd().getBucketCols().add(bucketFieldName);
-    }
-    
-    t.getSd().setSerdeInfo(new SerDeInfo());
-    t.getSd().getSerdeInfo().setName(t.getTableName());
-    t.getSd().getSerdeInfo().setSerializationClass(schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS)); 
-    t.getSd().getSerdeInfo().setSerializationFormat(schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT)); 
-    t.getSd().getSerdeInfo().setSerializationLib(schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB));
-    if(t.getSd().getSerdeInfo().getSerializationClass() == null || (t.getSd().getSerdeInfo().getSerializationClass().length() == 0)) {
-      t.getSd().getSerdeInfo().setSerializationClass(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_SERDE));
-    }
-    
-    // hack hack TODO:pc need to create a type and then associate the type name 
-    String colstr = schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS);
-    List<FieldSchema>  fields = new ArrayList<FieldSchema>();
-    t.getSd().setCols(fields);
-    if(colstr != null) {
-      String[] cols =  colstr.split(",");
-      for (String colName : cols) {
-        FieldSchema col = new FieldSchema(colName, Constants.STRING_TYPE_NAME, "default string type");
-        fields.add(col);
-      }
-    } 
-    
-    if(fields.size() == 0) {
-      fields.add(new FieldSchema("__SERDE__", t.getSd().getSerdeInfo().getSerializationLib(), ""));
-    }
-    
-    // finally create table
-    try {
-      this.createTable(t);
-    } catch (Exception e) {
-      MetaStoreUtils.logAndThrowMetaException(e);
-    }
-  }
-
   public List<String> getTables(String dbname, String tablePattern) throws MetaException {
     try {
       return client.get_tables(dbname, tablePattern);
@@ -543,4 +471,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     return getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
   }
 
+  @Override
+  public List<String> listPartitionNames(String dbName, String tblName, short max)
+      throws MetaException, TException {
+    // TODO Auto-generated method stub
+    return client.get_partition_names(dbName, tblName, max);
+  }
+
 }

+ 3 - 1
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java

@@ -87,9 +87,11 @@ public interface IMetaStoreClient {
    * @throws TException
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#list_partitions(java.lang.String, java.lang.String, short)
    */
-  public List<Partition> listPartitions(String tbl_name, String db_name, short max_parts)
+  public List<Partition> listPartitions(String db_name, String tbl_name, short max_parts)
       throws NoSuchObjectException, MetaException, TException;
 
+  public List<String> listPartitionNames(String db_name, String tbl_name, short max_parts)
+    throws  MetaException, TException;
   /**
    * @param tbl
    * @return

+ 10 - 3
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreClient.java

@@ -319,7 +319,7 @@ public class MetaStoreClient implements IMetaStoreClient {
 
   public Table getTable(String tableName) throws MetaException, TException, NoSuchObjectException {
     Properties schema = this.getSchema(tableName);
-    return MetaStoreUtils.getTable(schema);
+    return MetaStoreUtils.getTable(conf, schema);
   }
 
   //These will disappear when the server is unified for both filestore and dbstore
@@ -392,7 +392,7 @@ public class MetaStoreClient implements IMetaStoreClient {
   private Partition getPartitionObject(String dbName, String tableName, List<String> partVals)
       throws MetaException, TException, NoSuchObjectException {
     Properties schema = this.getSchema(tableName);
-    Table tbl = MetaStoreUtils.getTable(schema);
+    Table tbl = MetaStoreUtils.getTable(conf, schema);
     List<FieldSchema> partKeys = tbl.getPartitionKeys();
     if(partKeys.size() != partVals.size()) {
       throw new MetaException("Invalid partition key values: " + partVals);
@@ -407,7 +407,7 @@ public class MetaStoreClient implements IMetaStoreClient {
     Path partPath = wh.getPartitionPath(dbName, tableName, pm);
     Partition tPartition = new Partition();
     tPartition.setValues(partVals);
-    tPartition.setSd(tbl.getSd());
+    tPartition.setSd(tbl.getSd()); // TODO: get a copy
     tPartition.setParameters(new HashMap<String, String>());
     tPartition.getSd().setLocation(partPath.toString());
     return tPartition;
@@ -432,4 +432,11 @@ public class MetaStoreClient implements IMetaStoreClient {
     return false;
   }
 
+  @Override
+  public List<String> listPartitionNames(String db_name, String tbl_name, short max_parts)
+      throws MetaException, TException {
+    // TODO Auto-generated method stub
+    return new ArrayList<String>();
+  }
+
 }

+ 32 - 84
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreServer.java

@@ -27,8 +27,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Map.Entry;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -40,9 +38,12 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.ThriftMetaStore;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
-import org.apache.hadoop.hive.serde.SerDe;
-import org.apache.hadoop.hive.serde.SerDeException;
-import org.apache.hadoop.hive.serde.SerDeField;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.util.StringUtils;
 
 import com.facebook.fb303.FacebookBase;
@@ -298,66 +299,12 @@ public class MetaStoreServer extends ThriftMetaStore {
     }
 
 
-    /**
-     * normalizeType
-     *
-     * For pretty printing
-     *
-     * @param type a type name
-     * @return cleaned up - remove Java.lang and make numbers into int , ...
-     */
-    public String normalizeType(String type) {
-      Pattern tpat = Pattern.compile("java.lang.");
-      Matcher m = tpat.matcher(type);
-      type = m.replaceFirst("");
-      String ret = type;
-      if(type.equals("String")) {
-        ret = "string";
-      }  else if(type.equals("Integer")) {
-        ret = "int";
-      }
-      return ret;
-
-    }
-
-
-    /**
-     * hfToStrting
-     *
-     * Converts a basic SerDe's type to a string. It doesn't do any recursion.
-     *
-     * @param f the hive field - cannot be null!
-     * @return a string representation of the field's type
-     * @exception MetaException - if the SerDe raises it or the field is null
-     *
-     */
-    private String hfToString(SerDeField f) throws MetaException {
-      String ret;
-
-      try {
-        if(f.isPrimitive()) {
-          ret = this.normalizeType(f.getType().getName());
-        } else if(f.isList()) {
-          ret = "List<" +  this.normalizeType(f.getListElementType().getName()) + ">";
-        } else if(f.isMap()) {
-          ret = "Map<" + this.normalizeType(f.getMapKeyType().getName()) + "," +
-            this.normalizeType(f.getMapValueType().getName()) + ">";
-        } else {
-          // complex type and we just show the type name of the complex type
-          ret = f.getName();
-        }
-      }  catch(Exception e) {
-        StringUtils.stringifyException(e);
-        throw new MetaException(e.getMessage());
-      }
-      return ret;
-    }
     public ArrayList<FieldSchema> get_fields(String db, String table_name) throws MetaException,UnknownTableException, UnknownDBException {
 
       ArrayList<FieldSchema> str_fields = new ArrayList<FieldSchema>();
       String [] names = table_name.split("\\.");
       String base_table_name = names[0];
-      List<SerDeField> hive_fields = new ArrayList<SerDeField>();
+      String last_name = names[names.length-1];
 
       AbstractMap<String,String> schema_map = get_schema(base_table_name); // will throw UnknownTableException if not found
       Properties p = new Properties();
@@ -367,40 +314,41 @@ public class MetaStoreServer extends ThriftMetaStore {
       }
       // xxx
 
-      SerDeField hf = null;
-
       try {
         //            Table t = Table.readTable(p, this.db);
-        SerDe s = MetaStoreUtils.getSerDe( conf_, p);
-
+        Deserializer s = MetaStoreUtils.getDeserializer( conf_, p);
+        ObjectInspector oi = s.getObjectInspector();
+        
         // recurse down the type.subtype.subsubtype expression until at the desired type
         for(int i =  1; i < names.length; i++) {
-          hf = s.getFieldFromExpression(hf,names[i]);
+          if (!(oi instanceof StructObjectInspector)) {
+            oi = s.getObjectInspector();
+            break;
+          }
+          StructObjectInspector soi = (StructObjectInspector)oi;
+          StructField sf = soi.getStructFieldRef(names[i]);
+          if (sf == null) {
+            // If invalid field, then return the schema of the table
+            oi = s.getObjectInspector();
+            break;
+          } else {
+            oi = sf.getFieldObjectInspector();
+          }
         }
 
         // rules on how to recurse the SerDe based on its type
-        if(hf != null && hf.isPrimitive()) {
-          hive_fields.add(hf);
-        } else if(hf != null && hf.isList()) {
-          // don't remember why added this rule??
-          // should just be a hive_fields.add(hf)
-          try {
-            hive_fields = s.getFields(hf);
-          } catch(Exception e) {
-            hive_fields.add(hf);
-          }
-        } else if(hf != null && hf.isMap()) {
-          hive_fields.add(hf);
+        if (oi.getCategory() != Category.STRUCT) {
+          str_fields.add(new FieldSchema(last_name, oi.getTypeName(), "automatically generated"));
         } else {
-          hive_fields = s.getFields(hf);
-        }
-
-        for(SerDeField field: hive_fields) {
-          String name = field.getName();
-          String schema = this.hfToString(field);
-          str_fields.add(new FieldSchema(name, schema, "automatically generated"));
+          List<? extends StructField> fields = ((StructObjectInspector)oi).getAllStructFieldRefs();
+          for(int i=0; i<fields.size(); i++) {
+            String fieldName = fields.get(i).getFieldName();
+            String fieldTypeName = fields.get(i).getFieldObjectInspector().getTypeName();
+            str_fields.add(new FieldSchema(fieldName, fieldTypeName, "automatically generated"));
+          }
         }
         return str_fields;
+
       } catch(SerDeException e) {
         StringUtils.stringifyException(e);
         MetaException m = new MetaException();

+ 118 - 53
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java

@@ -41,8 +41,14 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.serde.SerDe;
-import org.apache.hadoop.hive.serde.SerDeUtils;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.util.StringUtils;
 
 public class MetaStoreUtils {
@@ -74,14 +80,17 @@ public class MetaStoreUtils {
     Table tTable = new Table();
     tTable.setTableName(name);
     tTable.setSd(new StorageDescriptor());
-    tTable.getSd().setSerdeInfo(new SerDeInfo());
-    tTable.getSd().getSerdeInfo().setSerializationLib(org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe.class.getName());
-    tTable.getSd().getSerdeInfo().setSerializationFormat("1");
+    StorageDescriptor sd = tTable.getSd();
+    sd.setSerdeInfo(new SerDeInfo());
+    SerDeInfo serdeInfo = sd.getSerdeInfo();
+    serdeInfo.setSerializationLib(MetadataTypedColumnsetSerDe.class.getName());
+    serdeInfo.setParameters(new HashMap<String, String>());
+    serdeInfo.getParameters().put(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
     
     List<FieldSchema>  fields = new ArrayList<FieldSchema>();
-    tTable.getSd().setCols(fields);
+    sd.setCols(fields);
     for (String col: columns) {
-      FieldSchema field = new FieldSchema(col, Constants.STRING_TYPE_NAME, "default string type");
+      FieldSchema field = new FieldSchema(col, org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "'default'");
       fields.add(field);
     }
 
@@ -89,12 +98,12 @@ public class MetaStoreUtils {
     for (String partCol : partCols) {
       FieldSchema part = new FieldSchema();
       part.setName(partCol);
-      part.setType(Constants.STRING_TYPE_NAME); // default partition key
+      part.setType(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME); // default partition key
       tTable.getPartitionKeys().add(part);
     }
     // not sure why these are needed
-    tTable.getSd().getSerdeInfo().setSerializationLib(org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe.shortName());
-    tTable.getSd().setNumBuckets(-1);
+    serdeInfo.setSerializationLib(MetadataTypedColumnsetSerDe.shortName());
+    sd.setNumBuckets(-1);
     return tTable;
   }
 
@@ -122,25 +131,25 @@ public class MetaStoreUtils {
 
 
   /**
-   * getSerDe
+   * getDeserializer
    *
-   * Get the SerDe for a table given its name and properties.
+   * Get the Deserializer for a table given its name and properties.
    *
    * @param name the name of the table
    * @param conf - hadoop config
    * @param p - the properties to use to instantiate the schema
-   * @return the SerDe
-   * @exception MetaException if any problems instantiating the serde
+   * @return the Deserializer
+   * @exception MetaException if any problems instantiating the Deserializer
    *
    * todo - this should move somewhere into serde.jar
    *
    */
-  static public SerDe getSerDe(Configuration conf, Properties schema) throws MetaException  {
+  static public Deserializer getDeserializer(Configuration conf, Properties schema) throws MetaException  {
     String lib = schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB);
     try {
-      SerDe serDe = SerDeUtils.lookupSerDe(lib);
-      ((SerDe)serDe).initialize(conf, schema);
-      return serDe;
+      Deserializer deserializer = SerDeUtils.lookupDeserializer(lib);
+      ((Deserializer)deserializer).initialize(conf, schema);
+      return deserializer;
     } catch (Exception e) {
       LOG.error("error in initSerDe: " + e.getClass().getName() + " " + e.getMessage());
       MetaStoreUtils.printStackTrace(e);
@@ -149,25 +158,25 @@ public class MetaStoreUtils {
   }
 
   /**
-   * getSerDe
+   * getDeserializer
    *
-   * Get the SerDe for a table given its name and properties.
+   * Get the Deserializer for a table given its name and properties.
    *
    * @param name the name of the table
    * @param conf - hadoop config
    * @param p - SerDe info
-   * @return the SerDe
-   * @exception MetaException if any problems instantiating the serde
+   * @return the Deserializer
+   * @exception MetaException if any problems instantiating the Deserializer
    *
    * todo - this should move somewhere into serde.jar
    *
    */
-  static public SerDe getSerDe(Configuration conf, org.apache.hadoop.hive.metastore.api.Table table) throws MetaException  {
+  static public Deserializer getDeserializer(Configuration conf, org.apache.hadoop.hive.metastore.api.Table table) throws MetaException  {
     String lib = table.getSd().getSerdeInfo().getSerializationLib();
     try {
-      SerDe serDe = SerDeUtils.lookupSerDe(lib);
-      ((SerDe)serDe).initialize(conf, MetaStoreUtils.getSchema(table));
-      return serDe;
+      Deserializer deserializer = SerDeUtils.lookupDeserializer(lib);
+      deserializer.initialize(conf, MetaStoreUtils.getSchema(table));
+      return deserializer;
     } catch (Exception e) {
       LOG.error("error in initSerDe: " + e.getClass().getName() + " " + e.getMessage());
       MetaStoreUtils.printStackTrace(e);
@@ -220,9 +229,9 @@ public class MetaStoreUtils {
 
 
   /**
-   * validateTableName
+   * validateName
    *
-   * Checks the name conforms to our standars which are: "[a-zA-z-_.0-9]+".
+   * Checks the name conforms to our standars which are: "[a-zA-z-_0-9]+".
    * checks this is just characters and numbers and _ and . and -
    *
    * @param tableName the name to validate
@@ -249,19 +258,22 @@ public class MetaStoreUtils {
     for(Enumeration<?> e = p.propertyNames(); e.hasMoreElements() ; ) {
       String key = (String)e.nextElement();
       String oldName = p.getProperty(key);
-
       oldName = oldName.replace("com.facebook.infrastructure.tprofiles","com.facebook.serde.tprofiles");
-
+      
       oldName = oldName.replace("com.facebook.infrastructure.hive_context","com.facebook.serde.hive_context");
+      oldName = oldName.replace("com.facebook.serde.hive_context","com.facebook.serde2.hive_context");
 
-      oldName = oldName.replace("com.facebook.thrift.hive.MetadataTypedColumnsetSerDe",org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe.class.getName());
+      oldName = oldName.replace("com.facebook.thrift.hive.MetadataTypedColumnsetSerDe",org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe.class.getName());
 
       // columnset serde
-      oldName = oldName.replace("com.facebook.thrift.hive.columnsetSerDe",org.apache.hadoop.hive.serde.thrift.columnsetSerDe.class.getName());
+      oldName = oldName.replace("com.facebook.thrift.hive.columnsetSerDe",org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe.class.getName());
+      oldName = oldName.replace("org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe",
+          org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe.class.getName());
 
       // thrift serde
-      oldName = oldName.replace("com.facebook.thrift.hive.ThriftHiveSerDe",org.apache.hadoop.hive.serde.thrift.ThriftSerDe.class.getName());
-
+      oldName = oldName.replace("com.facebook.thrift.hive.ThriftHiveSerDe",org.apache.hadoop.hive.serde2.ThriftDeserializer.class.getName());
+      oldName = oldName.replace("org.apache.hadoop.hive.serde.thrift.ThriftSerDe",
+          org.apache.hadoop.hive.serde2.ThriftDeserializer.class.getName());
       p.setProperty(key,oldName);
     }
     return p;
@@ -275,7 +287,7 @@ public class MetaStoreUtils {
     return "map<" + k +"," + v + ">";
   }
 
-  public static Table getTable(Properties schema) {
+  public static Table getTable(Configuration conf, Properties schema) throws MetaException {
     Table t = new Table();
     t.setSd(new StorageDescriptor());
     t.setTableName(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME));
@@ -285,7 +297,7 @@ public class MetaStoreUtils {
     t.getSd().setOutputFormat(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT,
           org.apache.hadoop.mapred.SequenceFileOutputFormat.class.getName())); 
     t.setPartitionKeys(new ArrayList<FieldSchema>());
-    t.setDatabase(MetaStoreUtils.DEFAULT_DATABASE_NAME);
+    t.setDbName(MetaStoreUtils.DEFAULT_DATABASE_NAME);
     String part_cols_str = schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS);
     t.setPartitionKeys(new ArrayList<FieldSchema>());
     if (part_cols_str != null && (part_cols_str.trim().length() != 0)) {
@@ -293,7 +305,7 @@ public class MetaStoreUtils {
       for (String key: part_keys) {
         FieldSchema part = new FieldSchema();
         part.setName(key);
-        part.setType(Constants.STRING_TYPE_NAME); // default partition key
+        part.setType(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME); // default partition key
         t.getPartitionKeys().add(part);
       }
     }
@@ -306,28 +318,37 @@ public class MetaStoreUtils {
     }
     
     t.getSd().setSerdeInfo(new SerDeInfo());
+    t.getSd().getSerdeInfo().setParameters(new HashMap<String, String>());
     t.getSd().getSerdeInfo().setName(t.getTableName());
-    t.getSd().getSerdeInfo().setSerializationClass(schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS)); 
-    t.getSd().getSerdeInfo().setSerializationFormat(schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT)); 
     t.getSd().getSerdeInfo().setSerializationLib(schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB));
-    if(t.getSd().getSerdeInfo().getSerializationClass() == null || (t.getSd().getSerdeInfo().getSerializationClass().length() == 0)) {
-      t.getSd().getSerdeInfo().setSerializationClass(schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_SERDE));
+    setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS);
+    setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT);
+    if(org.apache.commons.lang.StringUtils.isNotBlank(schema.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS))) {
+      setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_SERDE);
     }
+    // needed for MetadataTypedColumnSetSerDe
+    setSerdeParam(t.getSd().getSerdeInfo(), schema, org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS);
     
     String colstr = schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS);
     List<FieldSchema>  fields = new ArrayList<FieldSchema>();
-    t.getSd().setCols(fields);
     if(colstr != null) {
       String[] cols =  colstr.split(",");
       for (String colName : cols) {
-        FieldSchema col = new FieldSchema(colName, Constants.STRING_TYPE_NAME, "default string type");
+        FieldSchema col = new FieldSchema(colName, org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "'default'");
         fields.add(col);
       }
     } 
     
     if(fields.size() == 0) {
-      fields.add(new FieldSchema("__SERDE__", t.getSd().getSerdeInfo().getSerializationLib(), ""));
+      // get the fields from serde
+      try {
+        fields = getFieldsFromDeserializer(t.getTableName(), getDeserializer(conf, schema));
+      } catch (SerDeException e) {
+        LOG.error(StringUtils.stringifyException(e));
+        throw new MetaException("Invalid serde or schema. " + e.getMessage());
+      }
     }
+    t.getSd().setCols(fields);
     
     // remove all the used up parameters to find out the remaining parameters
     schema.remove(Constants.META_TABLE_NAME);
@@ -337,9 +358,9 @@ public class MetaStoreUtils {
     schema.remove(Constants.META_TABLE_PARTITION_COLUMNS);
     schema.remove(Constants.BUCKET_COUNT);
     schema.remove(Constants.BUCKET_FIELD_NAME);
-    schema.remove(Constants.SERIALIZATION_CLASS);
-    schema.remove(Constants.SERIALIZATION_FORMAT);
-    schema.remove(Constants.SERIALIZATION_LIB);
+    schema.remove(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS);
+    schema.remove(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT);
+    schema.remove(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB);
     schema.remove(Constants.META_TABLE_SERDE);
     schema.remove(Constants.META_TABLE_COLUMNS);
     
@@ -352,6 +373,13 @@ public class MetaStoreUtils {
     return t;
   }
 
+  private static void setSerdeParam(SerDeInfo sdi, Properties schema, String param) {
+    String val = schema.getProperty(param);
+    if(org.apache.commons.lang.StringUtils.isNotBlank(val)) {
+      sdi.getParameters().put(param, val);
+    }
+  }
+
   public static Properties getSchema(org.apache.hadoop.hive.metastore.api.Table tbl) {
     Properties schema = new Properties();
     String inputFormat = tbl.getSd().getInputFormat();
@@ -372,12 +400,7 @@ public class MetaStoreUtils {
     if (tbl.getSd().getBucketCols().size() > 0) {
       schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_FIELD_NAME, tbl.getSd().getBucketCols().get(0));
     }
-    if(tbl.getSd().getSerdeInfo().getSerializationClass() != null) {
-      schema.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS, tbl.getSd().getSerdeInfo().getSerializationClass());
-    }
-    if(tbl.getSd().getSerdeInfo().getSerializationFormat() != null) {
-      schema.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, tbl.getSd().getSerdeInfo().getSerializationFormat());
-    }
+    schema.putAll(tbl.getSd().getSerdeInfo().getParameters());
     if(tbl.getSd().getSerdeInfo().getSerializationLib() != null) {
       schema.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB, tbl.getSd().getSerdeInfo().getSerializationLib());
     }
@@ -440,4 +463,46 @@ public class MetaStoreUtils {
     LOG.error(StringUtils.stringifyException(e));
     throw new MetaException("Got exception: " + e.getClass().getName() + " " + e.getMessage());
   }
+
+  /**
+   * @param tableName
+   * @param deserializer
+   * @return
+   * @throws SerDeException
+   * @throws MetaException
+   */
+  static List<FieldSchema> getFieldsFromDeserializer(String tableName, Deserializer deserializer) throws SerDeException, MetaException {
+    ObjectInspector oi = deserializer.getObjectInspector();
+    String [] names = tableName.split("\\.");
+    String last_name = names[names.length-1];
+    for(int i = 1; i < names.length; i++) {
+      if (!(oi instanceof StructObjectInspector)) {
+        oi = deserializer.getObjectInspector();
+        break;
+      }
+      StructObjectInspector soi = (StructObjectInspector)oi;
+      StructField sf = soi.getStructFieldRef(names[i]);
+      if (sf == null) {
+        // If invalid field, then return the schema of the table
+        oi = deserializer.getObjectInspector();
+        break;
+      } else {
+        oi = sf.getFieldObjectInspector();
+      }
+    }
+
+    ArrayList<FieldSchema> str_fields = new ArrayList<FieldSchema>(); 
+    // rules on how to recurse the ObjectInspector based on its type
+    if (oi.getCategory() != Category.STRUCT) {
+      str_fields.add(new FieldSchema(last_name, oi.getTypeName(), "from deserializer"));
+    } else {
+      List<? extends StructField> fields = ((StructObjectInspector)oi).getAllStructFieldRefs();
+      for(int i=0; i<fields.size(); i++) {
+        String fieldName = fields.get(i).getFieldName();
+        String fieldTypeName = fields.get(i).getFieldObjectInspector().getTypeName();
+        str_fields.add(new FieldSchema(fieldName, fieldTypeName, "from deserializer"));
+      }
+    }
+    return str_fields;
+  }
 }

+ 57 - 44
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java

@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Map.Entry;
 
@@ -140,28 +141,31 @@ public class ObjectStore implements RawStore, Configurable {
     URL url= classLoader.getResource(JPOX_CONFIG);
     prop = new Properties();
     if (url == null) {
-      LOG.info(JPOX_CONFIG + " not found");
-      //throw new RuntimeException("Properties file not found " + JPOX_CONFIG);
+      LOG.info(JPOX_CONFIG + " not found.");
     } else {
       LOG.info("found resource " + JPOX_CONFIG + " at " + url);
       try {
         InputStream is = url.openStream();
         if (is == null) {
-          throw new RuntimeException("Properties file not found " + JPOX_CONFIG);
+          throw new RuntimeException("Properties file " + url + " couldn't be opened");
         }
-
         prop.load(is);
       } catch (IOException ex) {
         throw new RuntimeException("could not load: " + JPOX_CONFIG, ex);
       }
     }
-    addHiveConfProperty(javax.jdo.Constants.PROPERTY_CONNECTION_URL);
-    addHiveConfProperty(javax.jdo.Constants.PROPERTY_CONNECTION_DRIVER_NAME);
-    addHiveConfProperty(javax.jdo.Constants.PROPERTY_CONNECTION_USER_NAME);
-    addHiveConfProperty(javax.jdo.Constants.PROPERTY_CONNECTION_PASSWORD);
-    addHiveConfProperty(javax.jdo.Constants.PROPERTY_NONTRANSACTIONAL_READ);
-    addHiveConfProperty(javax.jdo.Constants.PROPERTY_DETACH_ALL_ON_COMMIT);
-    addHiveConfProperty(javax.jdo.Constants.PROPERTY_PERSISTENCE_MANAGER_FACTORY_CLASS);
+    
+    Iterator<Map.Entry<String, String>> iter = hiveConf.iterator();
+    while(iter.hasNext()) {
+      Map.Entry<String, String> e = iter.next();
+      if(e.getKey().contains("jpox") || e.getKey().contains("jdo")) {
+        Object prevVal = prop.setProperty(e.getKey(), e.getValue());
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Overriding " + e.getKey() + " value " + prevVal 
+              + " from  jpox.properties with " + e.getValue());
+        }
+      }
+    }
 
     if(LOG.isDebugEnabled()) {
       for (Entry<Object, Object> e: prop.entrySet()) {
@@ -170,15 +174,6 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
-  /**
-   * Overwrites the default jpox.properties attribute with what is specified in hive-default.xml
-   */
-  private void addHiveConfProperty(String param) {
-    String val = null;
-    if( (val=hiveConf.get(param)) != null) {
-      prop.setProperty(param, val);
-    }
-  }
   private static PersistenceManagerFactory getPMF() {
     if(pmf == null) {
       pmf = JDOHelper.getPersistenceManagerFactory(prop);
@@ -232,7 +227,7 @@ public class ObjectStore implements RawStore, Configurable {
    */
   @SuppressWarnings("nls")
   public boolean commitTransaction() {
-    assert(this.openTrasactionCalls > 1);
+    assert(this.openTrasactionCalls >= 1);
     if(!currentTransaction.isActive()) {
       throw new RuntimeException("Commit is called, but transaction is not active. Either there are" +
           "mismatching open and close calls or rollback was called in the same trasaction");
@@ -259,8 +254,10 @@ public class ObjectStore implements RawStore, Configurable {
    * Rolls back the current transaction if it is active
    */
   public void rollbackTransaction() {
-    assert(this.openTrasactionCalls > 1);
-    this.openTrasactionCalls--;
+    if(this.openTrasactionCalls < 1) {
+      return;
+    }
+    this.openTrasactionCalls = 0;
     if(currentTransaction.isActive() && transactionStatus != TXN_STATUS.ROLLBACK) {
       transactionStatus = TXN_STATUS.ROLLBACK;
        // could already be rolled back
@@ -271,7 +268,7 @@ public class ObjectStore implements RawStore, Configurable {
   public boolean createDatabase(Database db) {
     boolean success = false;
     boolean commited = false;
-    MDatabase mdb = new MDatabase(db.getName().toLowerCase(), db.getLocationUri());
+    MDatabase mdb = new MDatabase(db.getName().toLowerCase(), db.getDescription());
     try {
       openTransaction();
       pm.makePersistent(mdb);
@@ -326,7 +323,7 @@ public class ObjectStore implements RawStore, Configurable {
         rollbackTransaction();
       }
     }
-    return new Database(db.getName(), db.getLocation());
+    return new Database(db.getName(), db.getDescription());
   }
 
   public boolean dropDatabase(String dbname) {
@@ -587,10 +584,10 @@ public class ObjectStore implements RawStore, Configurable {
     if(tbl == null) return null;
     MDatabase mdb = null;
     try {
-      mdb = this.getMDatabase(tbl.getDatabase());
+      mdb = this.getMDatabase(tbl.getDbName());
     } catch (NoSuchObjectException e) {
       LOG.error(StringUtils.stringifyException(e));
-      throw new InvalidObjectException("Database " + tbl.getDatabase() + " doesn't exsit.");
+      throw new InvalidObjectException("Database " + tbl.getDbName() + " doesn't exsit.");
     }
     return new MTable(tbl.getTableName().toLowerCase(),
         mdb,
@@ -650,26 +647,14 @@ public class ObjectStore implements RawStore, Configurable {
   private SerDeInfo converToSerDeInfo(MSerDeInfo ms) throws MetaException {
    if(ms == null) throw new MetaException("Invalid SerDeInfo object");
    return new SerDeInfo(ms.getName(),
-       ms.getSerializationFormat(),
-       ms.getSerializationClass(),
        ms.getSerializationLib(),
-       ms.getFieldDelim(),
-       ms.getCollectionItemDelim(),
-       ms.getMapKeyDelim(),
-       ms.getLineDelim(),
        ms.getParameters()); 
   }
   
   private MSerDeInfo converToMSerDeInfo(SerDeInfo ms) throws MetaException {
     if(ms == null) throw new MetaException("Invalid SerDeInfo object");
     return new MSerDeInfo(ms.getName(),
-        ms.getSerializationFormat(),
-        ms.getSerializationClass(),
         ms.getSerializationLib(),
-        ms.getFieldDelim(),
-        ms.getCollectionItemDelim(),
-        ms.getMapKeyDelim(),
-        ms.getLineDelim(),
         ms.getParameters()); 
    }
   
@@ -697,7 +682,7 @@ public class ObjectStore implements RawStore, Configurable {
         sd.getLocation(),
         sd.getInputFormat(),
         sd.getOutputFormat(),
-        sd.isIsCompressed(),
+        sd.isCompressed(),
         sd.getNumBuckets(),
         converToMSerDeInfo(sd.getSerdeInfo()),
         sd.getBucketCols(),
@@ -762,7 +747,7 @@ public class ObjectStore implements RawStore, Configurable {
     if(part == null) {
       return null;
     }
-    MTable mt = getMTable(part.getDatabase(), part.getTableName());
+    MTable mt = getMTable(part.getDbName(), part.getTableName());
     if(mt == null) {
       throw new InvalidObjectException("Partition doesn't have a valid table or database name");
     }
@@ -822,6 +807,34 @@ public class ObjectStore implements RawStore, Configurable {
     return parts;
   }
 
+
+  //TODO:pc implement max
+  public List<String> listPartitionNames(String dbName, String tableName, short max) throws MetaException {
+    List<String> pns = new ArrayList<String>();
+    boolean success = false;
+    try {
+      openTransaction();
+      LOG.debug("Executing getPartitionNames");
+      dbName = dbName.toLowerCase();
+      tableName = tableName.toLowerCase();
+      Query q = pm.newQuery("select partitionName from org.apache.hadoop.hive.metastore.model.MPartition where table.database.name == t1 && table.tableName == t2");
+      q.declareParameters("java.lang.String t1, java.lang.String t2");
+      q.setResult("partitionName");
+      Collection names = (Collection) q.execute(dbName.trim(), tableName.trim());
+      pns = new ArrayList<String>(); 
+      for (Iterator i = names.iterator (); i.hasNext ();) {
+          pns.add((String) i.next ()); 
+      }
+      success = commitTransaction();
+    } finally {
+      if(!success) {
+        rollbackTransaction();
+        success = false;
+      }
+    }
+    return pns;
+  }
+  
   // TODO:pc implement max
   private List<MPartition> listMPartitions(String dbName, String tableName, int max) {
     boolean success = false;
@@ -853,11 +866,11 @@ public class ObjectStore implements RawStore, Configurable {
       openTransaction();
       name = name.toLowerCase();
       dbname = dbname.toLowerCase();
-      MTable newt = this.getMTable(newTable.getDatabase(), newTable.getTableName());
+      MTable newt = this.getMTable(newTable.getDbName(), newTable.getTableName());
       if(newt != null) {
-        if(!newTable.getTableName().equals(name) || !newTable.getDatabase().equals(dbname)) {
+        if(!newTable.getTableName().equals(name) || !newTable.getDbName().equals(dbname)) {
           // if the old table and new table aren't the same
-          throw new InvalidObjectException("new table " + newTable.getDatabase() +" already exists");
+          throw new InvalidObjectException("new table " + newTable.getDbName() +" already exists");
         }
       }
       newt = convertToMTable(newTable);

+ 2 - 0
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java

@@ -90,4 +90,6 @@ public interface RawStore extends Configurable {
       throws InvalidObjectException, MetaException;
 
   public List<String> getTables(String dbName, String pattern) throws MetaException;
+
+  public abstract List<String> listPartitionNames(String db_name, String tbl_name, short max_parts) throws MetaException;
 }

+ 1 - 1
src/contrib/hive/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java

@@ -144,7 +144,7 @@ public class Warehouse {
 
   public static String makePartName(List<FieldSchema> partCols, List<String> vals) throws MetaException {
     if ((partCols.size() != vals.size()) || (partCols.size() == 0)) {
-      throw new MetaException("Invalid key & values");
+      throw new MetaException("Invalid partition key & values");
     }
     StringBuilder name = new StringBuilder();
     for(int i=0; i< partCols.size(); i++) {

+ 9 - 8
src/contrib/hive/metastore/src/model/org/apache/hadoop/hive/metastore/model/MDatabase.java

@@ -27,7 +27,7 @@ package org.apache.hadoop.hive.metastore.model;
  */
 public class MDatabase {
   private String name;
-  private String location;
+  private String description;
 
   /**
    * Default construction to keep jpox/jdo happy
@@ -41,7 +41,7 @@ public class MDatabase {
    */
   public MDatabase(String name, String location) {
     this.name = name;
-    this.location = location;
+    this.description = location;
   }
 
   /**
@@ -59,16 +59,17 @@ public class MDatabase {
   }
 
   /**
-   * @return the location
+   * @return the description
    */
-  public String getLocation() {
-    return location;
+  public String getDescription() {
+    return description;
   }
 
   /**
-   * @param location the location to set
+   * @param description the description to set
    */
-  public void setLocation(String location) {
-    this.location = location;
+  public void setDescription(String description) {
+    this.description = description;
   }
+
 }

+ 1 - 105
src/contrib/hive/metastore/src/model/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java

@@ -22,68 +22,20 @@ import java.util.Map;
 
 public class MSerDeInfo {
   private String name;
-  private String serializationFormat;
-  private String serializationClass;
   private String serializationLib;
-  private String fieldDelim;
-  private String collectionItemDelim;
-  private String mapKeyDelim;
-  private String lineDelim;
   private Map<String, String> parameters;
 
   /**
    * @param name
-   * @param serializationFormat
-   * @param serializationClass
    * @param serializationLib
-   * @param fieldDelim
-   * @param collectionItemDelim
-   * @param mapKeyDelim
-   * @param lineDelim
    * @param parameters
    */
-  public MSerDeInfo(String name, String serializationFormat, String serializationClass,
-      String serializationLib, String fieldDelim, String collectionItemDelim, String mapKeyDelim,
-      String lineDelim, Map<String, String> parameters) {
+  public MSerDeInfo(String name, String serializationLib, Map<String, String> parameters) {
     this.name = name;
-    this.serializationFormat = serializationFormat;
-    this.serializationClass = serializationClass;
     this.serializationLib = serializationLib;
-    this.fieldDelim = fieldDelim;
-    this.collectionItemDelim = collectionItemDelim;
-    this.mapKeyDelim = mapKeyDelim;
-    this.lineDelim = lineDelim;
     this.parameters = parameters;
   }
 
-  /**
-   * @return the serializationFormat
-   */
-  public String getSerializationFormat() {
-    return serializationFormat;
-  }
-
-  /**
-   * @param serializationFormat the serializationFormat to set
-   */
-  public void setSerializationFormat(String serializationFormat) {
-    this.serializationFormat = serializationFormat;
-  }
-
-  /**
-   * @return the serializationClass
-   */
-  public String getSerializationClass() {
-    return serializationClass;
-  }
-
-  /**
-   * @param serializationClass the serializationClass to set
-   */
-  public void setSerializationClass(String serializationClass) {
-    this.serializationClass = serializationClass;
-  }
-
   /**
    * @return the serializationLib
    */
@@ -112,62 +64,6 @@ public class MSerDeInfo {
     this.name = name;
   }
 
-  /**
-   * @return the fieldDelim
-   */
-  public String getFieldDelim() {
-    return fieldDelim;
-  }
-
-  /**
-   * @param fieldDelim the fieldDelim to set
-   */
-  public void setFieldDelim(String fieldDelim) {
-    this.fieldDelim = fieldDelim;
-  }
-
-  /**
-   * @return the collectionItemDelim
-   */
-  public String getCollectionItemDelim() {
-    return collectionItemDelim;
-  }
-
-  /**
-   * @param collectionItemDelim the collectionItemDelim to set
-   */
-  public void setCollectionItemDelim(String collectionItemDelim) {
-    this.collectionItemDelim = collectionItemDelim;
-  }
-
-  /**
-   * @return the mapKeyDelim
-   */
-  public String getMapKeyDelim() {
-    return mapKeyDelim;
-  }
-
-  /**
-   * @param mapKeyDelim the mapKeyDelim to set
-   */
-  public void setMapKeyDelim(String mapKeyDelim) {
-    this.mapKeyDelim = mapKeyDelim;
-  }
-
-  /**
-   * @return the lineDelim
-   */
-  public String getLineDelim() {
-    return lineDelim;
-  }
-
-  /**
-   * @param lineDelim the lineDelim to set
-   */
-  public void setLineDelim(String lineDelim) {
-    this.lineDelim = lineDelim;
-  }
-
   /**
    * @return the parameters
    */

+ 78 - 45
src/contrib/hive/metastore/src/model/package.jdo

@@ -4,12 +4,15 @@
 <jdo>  
   <package name="org.apache.hadoop.hive.metastore.model">  
     <class name="MDatabase" identity-type="datastore" table="DBS" detachable="true">  
+      <datastore-identity>
+        <column name="DB_ID"/>
+      </datastore-identity>
       <field name="name">  
         <column name="NAME" length="128" jdbc-type="VARCHAR"/>  
         <index name="UniqueDatabase" unique="true"/>
       </field>
-      <field name="location">  
-        <column name="location" length="256" jdbc-type="VARCHAR"/>  
+      <field name="description">  
+        <column name="DESC" length="767" jdbc-type="VARCHAR"/>  
       </field>  
     </class>
 
@@ -18,10 +21,10 @@
         <column name="FNAME" length="128" jdbc-type="VARCHAR"/>
       </field>
       <field name="type" >
-        <column name="FTYPE" length="1024" jdbc-type="VARCHAR" allows-null="false"/>
+        <column name="FTYPE" length="767" jdbc-type="VARCHAR" allows-null="false"/>
       </field>
       <field name="comment" >
-        <column name="FCOMMENT" length="1024" jdbc-type="VARCHAR" allows-null="true"/>
+        <column name="FCOMMENT" length="767" jdbc-type="VARCHAR" allows-null="true"/>
       </field>
     </class>
 
@@ -59,24 +62,27 @@
     </class>
 
     <class name="MTable" table="TBLS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="TBL_ID"/>
+      </datastore-identity>
       <index name="UniqueTable" unique="true">
         <column name="TBL_NAME"/>
-        <column name="DATABASE_ID"/>
+        <column name="DB_ID"/>
       </index>
       <field name="tableName">
         <column name="TBL_NAME" length="128" jdbc-type="VARCHAR"/>
       </field>
       <field name="database">
-        <column name="DATABASE_ID"/>
+        <column name="DB_ID"/>
       </field>
       <field name="partitionKeys" table="PARTITION_KEYS" >
         <collection element-type="MFieldSchema"/>
         <join>
           <primary-key name="PARTITION_KEY_PK">
-            <column name="TABLE_ID"/>
+            <column name="TBL_ID"/>
             <column name="PKEY_NAME"/>
           </primary-key>
-          <column name="TABLE_ID"/>
+          <column name="TBL_ID"/>
         </join>
         <element>
           <embedded>
@@ -84,15 +90,17 @@
               <column name="PKEY_NAME" length="128" jdbc-type="VARCHAR"/>
               </field>
             <field name="type">
-              <column name="PKEY_TYPE" length="1024" jdbc-type="VARCHAR" allows-null="false"/>
+              <column name="PKEY_TYPE" length="767" jdbc-type="VARCHAR" allows-null="false"/>
             </field>
             <field name="comment" >
-              <column name="PKEY_COMMENT" length="1024" jdbc-type="VARCHAR" allows-null="true"/>
+              <column name="PKEY_COMMENT" length="767" jdbc-type="VARCHAR" allows-null="true"/>
             </field>
           </embedded>
         </element>
       </field>
-      <field name="sd" dependent="true"/>
+      <field name="sd" dependent="true">
+        <column name="SD_ID"/>
+      </field>
       <field name="owner">
         <column name="OWNER" length="128" jdbc-type="VARCHAR"/>
       </field>
@@ -107,53 +115,55 @@
       </field>
       <field name="parameters" table="TABLE_PARAMS">
         <map key-type="java.lang.String" value-type="java.lang.String"/>
-        <join/>
+        <join>
+          <column name="TBL_ID"/>
+        </join>
         <key>
            <column name="PARAM_KEY" length="256" jdbc-type="VARCHAR"/>
         </key>
         <value>
-           <column name="PARAM_VALUE" length="1024" jdbc-type="VARCHAR"/>
+           <column name="PARAM_VALUE" length="767" jdbc-type="VARCHAR"/>
         </value>
       </field>
     </class>
 
     <class name="MSerDeInfo" identity-type="datastore" table="SERDES" detachable="true">
+      <datastore-identity>
+        <column name="SERDE_ID"/>
+      </datastore-identity>
       <field name="name">
         <column name="NAME" length="128" jdbc-type="VARCHAR"/>
       </field>
-      <field name="serializationFormat">
-        <column name="SERIALIZATION_FORMAT" length="1024" jdbc-type="VARCHAR" allows-null="true"/>
-      </field>
-      <field name="serializationClass">
-        <column name="SERIALIZATION_CLASS" length="1024" jdbc-type="VARCHAR" allows-null="true"/>
-      </field>
       <field name="serializationLib">
-        <column name="SERIALIZATION_LIB" length="1024" jdbc-type="VARCHAR" allows-null="true"/>
-      </field>
-      <field name="fieldDelim">
-        <column name="FIELD_DELIM" length="128" jdbc-type="VARCHAR" allows-null="true"/>
-      </field>
-      <field name="collectionItemDelim">
-        <column name="COLLECTION_ITEM_DELIM" length="128" jdbc-type="VARCHAR" allows-null="true"/>
+        <column name="SLIB" length="767" jdbc-type="VARCHAR" allows-null="true"/>
       </field>
-      <field name="mapKeyDelim">
-        <column name="MAP_KEY_DELIM" length="128" jdbc-type="VARCHAR" allows-null="true"/>
-      </field>
-      <field name="lineDelim">
-        <column name="LINE_DELIM" length="128" jdbc-type="VARCHAR" allows-null="true"/>
+      <field name="parameters" table="SERDE_PARAMS">
+        <map key-type="java.lang.String" value-type="java.lang.String"/>
+        <join>
+          <column name="SERDE_ID"/>
+        </join>
+        <key>
+           <column name="PARAM_KEY" length="256" jdbc-type="VARCHAR"/>
+        </key>
+        <value>
+           <column name="PARAM_VALUE" length="767" jdbc-type="VARCHAR"/>
+        </value>
       </field>
     </class>
 
     <class name="MOrder" embedded-only="true" table="SORT_ORDER" detachable="true">
       <field name="col">
-        <column name="COLUMN_NAME" length="128" jdbc-type="VARCHAR"/>
+        <column name="COL_NAME" length="128" jdbc-type="VARCHAR"/>
       </field>
       <field name="order">
         <column name="ORDER" jdbc-type="INTEGER"  allows-null="false"/>
       </field>
     </class>
 
-    <class name="MStorageDescriptor" identity-type="datastore" table="STORAGE_DESC" detachable="true">
+    <class name="MStorageDescriptor" identity-type="datastore" table="SDS" detachable="true">
+      <datastore-identity>
+        <column name="SD_ID"/>
+      </datastore-identity>
       <field name="cols" table="COLUMNS" >
         <collection element-type="MFieldSchema"/>
         <join>
@@ -161,6 +171,7 @@
             <column name="COLUMN_NAME"/>
             <column name="TYPE_NAME"/>
           </primary-key>
+          <column name="SD_ID"/>
         </join>
         <element>
           <embedded>
@@ -174,7 +185,7 @@
         </element>
       </field>
       <field name="location">
-        <column name="LOCATION" length="1024" jdbc-type="VARCHAR"/>
+        <column name="LOCATION" length="767" jdbc-type="VARCHAR"/>
       </field>
       <field name="inputFormat">
         <column name="INPUT_FORMAT" length="256" jdbc-type="VARCHAR"/>
@@ -186,15 +197,21 @@
       <field name="numBuckets">
         <column name="NUM_BUCKETS" jdbc-type="integer"/>
       </field>
-      <field name="serDeInfo" dependent="true"/>
+      <field name="serDeInfo" dependent="true">
+        <column name="SERDE_ID"/>
+      </field>
       <field name="bucketCols" table="BUCKETING_COLS">
         <collection element-type="java.lang.String"/>
-        <join/>
+        <join>
+          <column name="SD_ID"/>
+        </join>
         <element column="BUCKET_COL_NAME"/>
       </field>
       <field name="sortCols" table="SORT_COLS">
         <collection element-type="MOrder"/>
-        <join/>
+        <join>
+          <column name="SD_ID"/>
+        </join>
         <element>
           <embedded>
             <field name="col">
@@ -208,21 +225,32 @@
       </field>
       <field name="parameters" table="SD_PARAMS">
         <map key-type="java.lang.String" value-type="java.lang.String"/>
-        <join/>
+        <join>
+          <column name="SD_ID"/>
+        </join>
         <key>
            <column name="PARAM_KEY" length="256" jdbc-type="VARCHAR"/>
         </key>
         <value>
-           <column name="PARAM_VALUE" length="1024" jdbc-type="VARCHAR"/>
+           <column name="PARAM_VALUE" length="767" jdbc-type="VARCHAR"/>
         </value>
       </field>
     </class>
 
     <class name="MPartition" table="PARTITIONS" identity-type="datastore" detachable="true">
+      <index name="UniquePartition" unique="true">
+        <column name="PART_NAME"/>
+        <column name="TBL_ID"/>
+      </index>
+      <datastore-identity>
+        <column name="PART_ID"/>
+      </datastore-identity>
       <field name="partitionName">
-        <column name="PART_NAME" length="1024" jdbc-type="VARCHAR"/>
+        <column name="PART_NAME" length="767" jdbc-type="VARCHAR"/>
+      </field>
+      <field name="table">
+        <column name="TBL_ID"/>
       </field>
-      <field name="table"/>
       <field name="createTime">
         <column name="CREATE_TIME" jdbc-type="integer"/>
       </field>
@@ -231,19 +259,24 @@
       </field>
       <field name="values" table="PARTITION_KEY_VALS">
         <collection element-type="java.lang.String"/>
-        <join/>
-        <element column="PARTITION_KEY_VAL"/>
+        <join>
+          <column name="PART_ID"/>
+        </join>
+        <element column="PART_KEY_VAL"/>
       </field>
       <field name="sd" dependent="true">
+        <column name="SD_ID"/>
       </field>
       <field name="parameters" table="PARTITION_PARAMS">
         <map key-type="java.lang.String" value-type="java.lang.String"/>
-        <join/>
+        <join>
+          <column name="PART_ID"/>
+        </join>
         <key>
            <column name="PARAM_KEY" length="256" jdbc-type="VARCHAR"/>
         </key>
         <value>
-           <column name="PARAM_VALUE" length="1024" jdbc-type="VARCHAR"/>
+           <column name="PARAM_VALUE" length="767" jdbc-type="VARCHAR"/>
         </value>
       </field>
     </class>

+ 1 - 1
src/contrib/hive/metastore/src/test/org/apache/hadoop/hive/metastore/MetaStoreTestBase.java

@@ -101,7 +101,7 @@ public class MetaStoreTestBase extends TestCase {
     Properties schema = new Properties();
     schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,name);
     schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_DB, db);
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_SERDE, org.apache.hadoop.hive.metastore.api.Constants.META_SERDE);
+    //schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_SERDE, org.apache.hadoop.hive.metastore.api.Constants.META_SERDE);
     schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS, "foo,bar");
     schema.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT,"\t");
     //Path tPath = whRoot_.suffix("/" + db + ".db/" + name);

+ 26 - 22
src/contrib/hive/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java

@@ -25,7 +25,7 @@ import java.util.List;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Constants;
+import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
@@ -91,12 +91,12 @@ public class TestHiveMetaStore extends TestCase {
     assertTrue("Unable to create type " + typeName, ret);
   
     Table tbl = new Table();
-    tbl.setDatabase(dbName);
+    tbl.setDbName(dbName);
     tbl.setTableName(tblName);
     StorageDescriptor sd = new StorageDescriptor(); 
     tbl.setSd(sd);
     sd.setCols(typ1.getFields());
-    sd.setIsCompressed(false);
+    sd.setCompressed(false);
     sd.setNumBuckets(1);
     sd.setParameters(new HashMap<String, String>());
     sd.getParameters().put("test_param_1", "Use this for comments etc");
@@ -104,7 +104,8 @@ public class TestHiveMetaStore extends TestCase {
     sd.getBucketCols().add("name");
     sd.setSerdeInfo(new SerDeInfo());
     sd.getSerdeInfo().setName(tbl.getTableName());
-    sd.getSerdeInfo().setSerializationFormat("3");
+    sd.getSerdeInfo().setParameters(new HashMap<String, String>());
+    sd.getSerdeInfo().getParameters().put(Constants.SERIALIZATION_FORMAT, "1");
     sd.setSortCols(new ArrayList<Order>());
   
     tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
@@ -114,7 +115,7 @@ public class TestHiveMetaStore extends TestCase {
     client.createTable(tbl);
   
     Partition part = new Partition();
-    part.setDatabase(dbName);
+    part.setDbName(dbName);
     part.setTableName(tblName);
     part.setValues(vals);
     part.setParameters(new HashMap<String, String>());
@@ -160,7 +161,7 @@ public class TestHiveMetaStore extends TestCase {
     Database db = client.getDatabase("test1");
     
     assertEquals("name of returned db is different from that of inserted db", "test1", db.getName());
-    assertEquals("location of the returned db is different from that of inserted db", "strange_loc", db.getLocationUri());
+    assertEquals("location of the returned db is different from that of inserted db", "strange_loc", db.getDescription());
     
     boolean ret2 = client.createDatabase("test2", "another_strange_loc");
     assertTrue("Unable to create the databse", ret2);
@@ -168,7 +169,7 @@ public class TestHiveMetaStore extends TestCase {
     Database db2 = client.getDatabase("test2");
     
     assertEquals("name of returned db is different from that of inserted db", "test2", db2.getName());
-    assertEquals("location of the returned db is different from that of inserted db", "another_strange_loc", db2.getLocationUri());
+    assertEquals("location of the returned db is different from that of inserted db", "another_strange_loc", db2.getDescription());
     
     List<String> dbs = client.getDatabases();
     
@@ -288,12 +289,12 @@ public class TestHiveMetaStore extends TestCase {
     assertTrue("Unable to create type " + typeName, ret);
     
     Table tbl = new Table();
-    tbl.setDatabase(dbName);
+    tbl.setDbName(dbName);
     tbl.setTableName(tblName);
     StorageDescriptor sd = new StorageDescriptor();
     tbl.setSd(sd);
     sd.setCols(typ1.getFields());
-    sd.setIsCompressed(false);
+    sd.setCompressed(false);
     sd.setNumBuckets(1);
     sd.setParameters(new HashMap<String, String>());
     sd.getParameters().put("test_param_1", "Use this for comments etc");
@@ -301,30 +302,32 @@ public class TestHiveMetaStore extends TestCase {
     sd.getBucketCols().add("name");
     sd.setSerdeInfo(new SerDeInfo());
     sd.getSerdeInfo().setName(tbl.getTableName());
-    sd.getSerdeInfo().setSerializationFormat("1");
+    sd.getSerdeInfo().setParameters(new HashMap<String, String>());
+    sd.getSerdeInfo().getParameters().put(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
     
     client.createTable(tbl);
     
     Table tbl2 = client.getTable(dbName, tblName);
     assertNotNull(tbl2);
-    assertEquals(tbl2.getDatabase(), dbName);
+    assertEquals(tbl2.getDbName(), dbName);
     assertEquals(tbl2.getTableName(), tblName);
     assertEquals(tbl2.getSd().getCols().size(), typ1.getFields().size());
-    assertEquals(tbl2.getSd().isIsCompressed(), false);
+    assertEquals(tbl2.getSd().isCompressed(), false);
     assertEquals(tbl2.getSd().getNumBuckets(), 1);
     assertEquals(tbl2.getSd().getLocation(), tbl.getSd().getLocation());
     assertNotNull(tbl2.getSd().getSerdeInfo());
-    assertEquals(tbl2.getSd().getSerdeInfo().getSerializationFormat(), "1");
+    sd.getSerdeInfo().setParameters(new HashMap<String, String>());
+    sd.getSerdeInfo().getParameters().put(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
     
     tbl.setTableName(tblName2);
     client.createTable(tbl);
   
     Table tbl3 = client.getTable(dbName, tblName2);
     assertNotNull(tbl3);
-    assertEquals(tbl3.getDatabase(), dbName);
+    assertEquals(tbl3.getDbName(), dbName);
     assertEquals(tbl3.getTableName(), tblName2);
     assertEquals(tbl3.getSd().getCols().size(), typ1.getFields().size());
-    assertEquals(tbl3.getSd().isIsCompressed(), false);
+    assertEquals(tbl3.getSd().isCompressed(), false);
     assertEquals(tbl3.getSd().getNumBuckets(), 1);
     assertEquals(tbl3.getSd().getLocation(), tbl.getSd().getLocation());
     
@@ -369,12 +372,12 @@ public class TestHiveMetaStore extends TestCase {
       assertTrue("Unable to create type " + typeName, ret);
   
       Table tbl = new Table();
-      tbl.setDatabase(dbName);
+      tbl.setDbName(dbName);
       tbl.setTableName(tblName);
       StorageDescriptor sd = new StorageDescriptor();
       tbl.setSd(sd);
       sd.setCols(typ1.getFields());
-      sd.setIsCompressed(false);
+      sd.setCompressed(false);
       sd.setNumBuckets(1);
       sd.setParameters(new HashMap<String, String>());
       sd.getParameters().put("test_param_1", "Use this for comments etc");
@@ -382,19 +385,20 @@ public class TestHiveMetaStore extends TestCase {
       sd.getBucketCols().add("name");
       sd.setSerdeInfo(new SerDeInfo());
       sd.getSerdeInfo().setName(tbl.getTableName());
-      sd.getSerdeInfo().setSerializationFormat("9");
+      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
+      sd.getSerdeInfo().getParameters().put(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "9");
   
       tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
-      tbl.getPartitionKeys().add(new FieldSchema("ds", Constants.DATE_TYPE_NAME, ""));
-      tbl.getPartitionKeys().add(new FieldSchema("hr", Constants.INT_TYPE_NAME, ""));
+      tbl.getPartitionKeys().add(new FieldSchema("ds", org.apache.hadoop.hive.serde.Constants.DATE_TYPE_NAME, ""));
+      tbl.getPartitionKeys().add(new FieldSchema("hr", org.apache.hadoop.hive.serde.Constants.INT_TYPE_NAME, ""));
   
       client.createTable(tbl);
   
       Table tbl2 = client.getTable(dbName, tblName);
-      assertEquals(tbl2.getDatabase(), dbName);
+      assertEquals(tbl2.getDbName(), dbName);
       assertEquals(tbl2.getTableName(), tblName);
       assertEquals(tbl2.getSd().getCols().size(), typ1.getFields().size());
-      assertFalse(tbl2.getSd().isIsCompressed());
+      assertFalse(tbl2.getSd().isCompressed());
       assertEquals(tbl2.getSd().getNumBuckets(), 1);
   
       assertEquals("Use this for comments etc", tbl2.getSd().getParameters().get("test_param_1"));

+ 10 - 17
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/Context.java

@@ -18,24 +18,20 @@
 
 package org.apache.hadoop.hive.ql;
 
-import java.io.File;
+import java.io.DataInput;
 import java.io.IOException;
 import java.io.FileNotFoundException;
-import java.io.InputStream;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.util.StringUtils;
 
 public class Context {
-  private File resFile;
+  private Path resFile;
   private Path resDir;
   private FileSystem fs;
   static final private Log LOG = LogFactory.getLog("hive.ql.Context");
@@ -57,14 +53,14 @@ public class Context {
   /**
    * @return the resFile
    */
-  public File getResFile() {
+  public Path getResFile() {
     return resFile;
   }
 
   /**
    * @param resFile the resFile to set
    */
-  public void setResFile(File resFile) {
+  public void setResFile(Path resFile) {
     this.resFile = resFile;
     resDir = null;
     resDirPaths = null;
@@ -105,10 +101,7 @@ public class Context {
     {
       try
       {
-        FileOutputStream outStream = new FileOutputStream(resFile);
-        outStream.close();
-      } catch (FileNotFoundException e) {
-        LOG.info("Context clear error: " + StringUtils.stringifyException(e));
+      	fs.delete(resFile, false);
       } catch (IOException e) {
         LOG.info("Context clear error: " + StringUtils.stringifyException(e));
       }
@@ -120,7 +113,7 @@ public class Context {
     resDirPaths = null;
   }
 
-  public InputStream getStream() {
+  public DataInput getStream() {
     try
     {
       if (!initialized) {
@@ -128,7 +121,7 @@ public class Context {
         if ((resFile == null) && (resDir == null)) return null;
       
         if (resFile != null)
-          return (InputStream)(new FileInputStream(resFile));
+          return (DataInput)fs.open(resFile);
         
         FileStatus status = fs.getFileStatus(resDir);
         assert status.isDir();
@@ -140,7 +133,7 @@ public class Context {
             resDirPaths[pos++] = resFS.getPath();
         if (pos == 0) return null;
         
-        return (InputStream)fs.open(resDirPaths[resDirFilesNum++]);
+        return (DataInput)fs.open(resDirPaths[resDirFilesNum++]);
       }
       else {
         return getNextStream();
@@ -154,12 +147,12 @@ public class Context {
     }
   }
 
-  private InputStream getNextStream() {
+  private DataInput getNextStream() {
     try
     {
       if (resDir != null && resDirFilesNum < resDirPaths.length && 
           (resDirPaths[resDirFilesNum] != null))
-        return (InputStream)fs.open(resDirPaths[resDirFilesNum++]);
+        return (DataInput)fs.open(resDirPaths[resDirFilesNum++]);
     } catch (FileNotFoundException e) {
       LOG.info("getNextStream error: " + StringUtils.stringifyException(e));
       return null;

+ 57 - 91
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/Driver.java

@@ -18,7 +18,8 @@
 
 package org.apache.hadoop.hive.ql;
 
-import java.io.InputStream;
+import java.io.DataInput;
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.*;
 import org.antlr.runtime.tree.CommonTree;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.MapRedTask;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.ExecDriver;
+import org.apache.hadoop.hive.serde.ByteStream;
 import org.apache.hadoop.hive.conf.HiveConf;
 
 import org.apache.commons.logging.Log;
@@ -45,23 +47,14 @@ import org.apache.commons.logging.LogFactory;
 public class Driver implements CommandProcessor {
 
   static final private Log LOG = LogFactory.getLog("hive.ql.Driver");
-  static final private int separator  = Utilities.ctrlaCode;
-  static final private int terminator = Utilities.newLineCode;
   static final private int MAX_ROWS   = 100;
+  ByteStream.Output bos = new ByteStream.Output();
   
-  private ParseDriver pd;
-  private HiveConf conf;
-  private InputStream resStream;
-  private LogHelper console;
-  private Context   ctx;
-  
-  public static int getSeparator() {
-    return separator;
-  }
-
-  public static int getTerminator() {
-    return terminator;
-  }
+  private ParseDriver  pd;
+  private HiveConf     conf;
+  private DataInput    resStream;
+  private LogHelper    console;
+  private Context      ctx;
   
   public int countJobs(List<Task<? extends Serializable>> tasks) {
     if (tasks == null)
@@ -108,12 +101,9 @@ public class Driver implements CommandProcessor {
       BaseSemanticAnalyzer sem;
       LOG.info("Starting command: " + command);
 
-      if (resStream != null) {
-        resStream.close();
-        resStream = null;
-      }
       ctx.clear();
-
+      resStream = null;
+      
       pd = new ParseDriver();
       CommonTree tree = pd.parse(command);
 
@@ -200,95 +190,71 @@ public class Driver implements CommandProcessor {
         conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, "");
       } 
     }
-    if (jobs > 0) {
-      console.printInfo("OK");
-    }
+
+    console.printInfo("OK");
     return (0);
   }
   
   
-  public boolean getResults(Vector<Vector<String>> res) {
-
+  public boolean getResults(Vector<Vector<String>> res) 
+  {
     if (resStream == null)
       resStream = ctx.getStream();
     if (resStream == null) return false;
     
-    int sizeArr = 128;
-    char[] tmpCharArr = new char[sizeArr];
-    
-    for (int numRows = 0; numRows < MAX_ROWS; numRows++)
+    int numRows = 0;
+    Vector<String> row = new Vector<String>();
+
+    while (numRows < MAX_ROWS)
     {
-      if (resStream == null) {
-        if (numRows > 0) {
+      if (resStream == null) 
+      {
+        if (numRows > 0)
           return true;
-        }
-        else {
+        else
           return false;
-        }
       }
-      boolean eof = false;
-      Vector<String> row = new Vector<String>();
-      String col;
-      int len = 0;
-      while (true) {
-        char c;
-        try {
-          int i = resStream.read();
-          if (i == -1)
-          {
-            eof = true;
-            break;
-          }
-          
-          c = (char)i;
-          
-          if (c == terminator) {
-            col = new String(tmpCharArr, 0, len);
-            len = 0;
-            row.add(col.equals(Utilities.nullStringStorage) ? null : col);
-            res.add(row);
-            break;
-          }
-          else if (c == separator) {
-            col = new String(tmpCharArr, 0, len);
-            len = 0;
-            row.add(col.equals(Utilities.nullStringStorage) ? null : col);
-          }
-          else
-          {
-            if (sizeArr == len)
-            {
-              char[] tmp = new char[2*sizeArr];
-              sizeArr *= 2;
-              for (int idx = 0; idx < len; idx++)
-                tmp[idx] = tmpCharArr[idx];
-              tmpCharArr = tmp;
-            }
-            tmpCharArr[len++] = c;
-          }
-          
-        } 
-        catch (java.io.IOException e) {
-          console.printError("FAILED: Unknown exception : " + e.getMessage(),
-                             "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
-          return false;
-        }
+
+      String col = null;
+      bos.reset();
+      Utilities.streamStatus ss = Utilities.streamStatus.NORMAL;
+      try
+      {
+        ss = Utilities.readColumn(resStream, bos);
+        if (bos.getCount() > 0)
+          col = new String(bos.getData(), 0, bos.getCount(), "UTF-8");
+        else if (ss == Utilities.streamStatus.NORMAL)
+          col = Utilities.NSTR;
+      } catch (IOException e) {
+        console.printError("FAILED: Unexpected IO exception : " + e.getMessage());
+        res = null;
+        return false;
       }
       
-      if (eof)
+      if ((ss == Utilities.streamStatus.EOF) || 
+          (ss == Utilities.streamStatus.TERMINATED))
       {
-        if (len > 0)
-        {
-          col = new String(tmpCharArr, 0, len);
-          len = 0;
+        if (col != null) 
           row.add(col.equals(Utilities.nullStringStorage) ? null : col);
-          res.add(row);
-        }
+        else if (row.size() != 0) 
+          row.add(null);
+
+        numRows++;
+        res.add(row);
+        row = new Vector<String>();
+        col = null;
 
-        resStream = ctx.getStream();
+        if (ss == Utilities.streamStatus.EOF) 
+          resStream = ctx.getStream();
       }
+      else if (ss == Utilities.streamStatus.NORMAL)
+      {
+        row.add(col.equals(Utilities.nullStringStorage) ? null : col);
+        col = null;
+      }
+      else
+        assert false;
     }
-    
     return true;
   }
 }

+ 28 - 10
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/CollectOperator.java

@@ -23,6 +23,9 @@ import java.io.*;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.collectDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -31,26 +34,41 @@ import org.apache.hadoop.conf.Configuration;
 public class CollectOperator extends Operator <collectDesc> implements Serializable {
 
   private static final long serialVersionUID = 1L;
-  transient protected ArrayList<HiveObject> objList;
+  transient protected ArrayList<Object> rowList;
+  transient protected ArrayList<ObjectInspector> rowInspectorList;
   transient int maxSize;
 
   public void initialize(Configuration hconf) throws HiveException {
     super.initialize(hconf);
-    objList = new ArrayList<HiveObject> ();
+    rowList = new ArrayList<Object> ();
+    rowInspectorList = new ArrayList<ObjectInspector> ();
     maxSize = conf.getBufferSize().intValue();
   }
 
-  public void process(HiveObject r) throws HiveException {
-    if(objList.size() < maxSize) {
-      objList.add(r);
+  public void process(Object row, ObjectInspector rowInspector)
+      throws HiveException {
+    if(rowList.size() < maxSize) {
+      // Create a standard copy of the object.
+      // In the future we can optimize this by doing copy-on-write.
+      // Here we always copy the object so that other operators can reuse the object for the next row. 
+      Object o = ObjectInspectorUtils.getStandardObject(row, rowInspector);
+      ObjectInspector oi = ObjectInspectorUtils.getStandardObjectInspector(rowInspector);
+      rowList.add(o);
+      rowInspectorList.add(oi);
     }
-    forward(r);
+    forward(row, rowInspector);
   }
   
-  public HiveObject retrieve() {
-    if(objList.isEmpty())
-      return null;
-    return objList.remove(0);
+  public void retrieve(InspectableObject result) {
+    assert(result != null);
+    if (rowList.isEmpty()) {
+      result.o = null;
+      result.oi = null;
+    } else {
+      result.o = rowList.remove(0);
+      result.oi = rowInspectorList.remove(0);
+    }
   }
 
+
 }

+ 3 - 2
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java

@@ -21,7 +21,8 @@ package org.apache.hadoop.hive.ql.exec;
 import java.lang.Class;
 import java.io.*;
 
-import org.apache.hadoop.hive.ql.parse.TypeInfo;
+import org.apache.hadoop.hive.ql.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.ql.typeinfo.TypeInfoFactory;
 
 /**
  * Implementation for ColumnInfo which contains the internal name for the 
@@ -55,7 +56,7 @@ public class ColumnInfo implements Serializable {
   
   public ColumnInfo(String internalName, Class type, boolean isVirtual) {
     this.internalName = internalName;
-    this.type = TypeInfo.getPrimitiveTypeInfo(type);
+    this.type = TypeInfoFactory.getPrimitiveTypeInfo(type);
     this.isVirtual = isVirtual;
   }
   

+ 0 - 178
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/CompositeHiveObject.java

@@ -1,178 +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.ql.exec;
-
-import java.util.*;
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * fixed width composition of Hive Objects
- */
-public class CompositeHiveObject extends HiveObject {
-
-  public static class CompositeSerDeField implements SerDeField {
-    int position;
-    String expr;
-    protected SerDeField field;
-
-    public CompositeSerDeField (int position, String expr) {
-      this.position = position;
-      this.expr = expr;
-    }
-
-    public HiveObject getHiveObject(Object obj) throws SerDeException {
-      ArrayList<HiveObject>  hol = (ArrayList<HiveObject>)obj;
-      HiveObject ho = hol.get(this.position);
-
-      if(ho == null)
-        return new NullHiveObject();
-      
-      if(this.expr == null) {
-        // no need to descend any further
-        return ho;
-      }
-
-      try {
-        if (this.field == null) {
-          this.field = ho.getFieldFromExpression(this.expr);
-        }
-        // descend recursively
-        return (ho.get(this.field));
-      } catch (HiveException e) {
-        throw new SerDeException (e);
-      }
-    }
-
-    public Object get(Object obj) throws SerDeException {
-      throw new RuntimeException ("get() cannot be called for CompositeSerDeField");
-    }
-    public boolean isList() { throw new RuntimeException("Not supported"); }
-    public boolean isMap() { throw new RuntimeException("Not supported"); }
-    public boolean isPrimitive() { throw new RuntimeException("Not supported"); }
-    public Class getType() { throw new RuntimeException("Not supported"); }
-    public Class getListElementType() {throw new RuntimeException("Not supported"); }
-    public Class getMapKeyType() { throw new RuntimeException("Not supported"); }
-    public Class getMapValueType() { throw new RuntimeException("Not supported"); }
-    public String getName() {  throw new RuntimeException("Not supported"); }
-  }
-
-  ArrayList<HiveObject> hol;
-  SerDe csd;
-  int width;
-
-  public CompositeHiveObject(int width) {
-    assert(width > 0);
-    this.hol = new ArrayList<HiveObject> (width);
-    this.width = width;
-  }
-
-  public CompositeHiveObject(ArrayList<HiveObject> hol) {
-    this.width = hol.size();
-    this.hol = hol;
-  }
-  
-  @Override
-  public SerDeField getFieldFromExpression(String compositeExpr) throws HiveException {
-    int position = this.width+1;
-    String expr = null;
-    int dot = compositeExpr.indexOf(".");
-    try {
-      if(dot == -1) {
-        position = Integer.parseInt(compositeExpr);
-        expr = null;
-      } else {
-        position = Integer.parseInt(compositeExpr.substring(0, dot));
-        expr = compositeExpr.substring(dot+1);
-        if(expr.equals("")) { expr = null; }
-      }
-    } catch (NumberFormatException e) {
-      throw new HiveException("Field Name is not a number: "+compositeExpr);
-    }
-    
-    if((position >= this.width) || (position < 0)) {
-      throw new HiveException("Index=" + position + " is not between 0 and (width=)" + this.width);
-    }
-    
-    return new CompositeSerDeField(position, expr);
-  }
-
-  @Override
-  public HiveObject get(SerDeField field) throws HiveException {
-    try {
-      CompositeSerDeField csdField = (CompositeSerDeField)field;
-      return (csdField.getHiveObject(this.hol));
-    } catch (SerDeException e) {
-      throw new HiveException(e);
-    }
-  }
-
-  @Override
-  public List<SerDeField> getFields() throws HiveException {
-    ArrayList<SerDeField> ret = new ArrayList<SerDeField> (this.width);
-    for(int i=0; i<this.width; i++) {
-      ret.add(new CompositeSerDeField(i, null));
-    }
-    return (ret);
-  }
-
-  public static List<SerDeField> getFields(int width) throws HiveException {
-    ArrayList<SerDeField> ret = new ArrayList<SerDeField> (width);
-    for(int i=0; i<width; i++) {
-      ret.add(new CompositeSerDeField(i, null));
-    }
-    return (ret);
-  }
-
-  @SuppressWarnings("nls")
-  public void addHiveObject(HiveObject ho) throws HiveException {
-    if(this.hol.size() >= this.width) {
-      throw new HiveException("Exceeded max size of Composite Hive Object ="+this.width);
-    }
-    this.hol.add(ho);
-  }
-
-  @Override
-  public Object getJavaObject() throws HiveException {
-    ArrayList result = new ArrayList();
-    for(HiveObject ho: hol) {
-      result.add(ho == null ? null : ho.getJavaObject());
-    }
-    return result;
-  }
-
-  @Override
-  public boolean isPrimitive() { return false; }
-
-  @Override
-  public boolean equals(Object other) {
-    if (!(other instanceof CompositeHiveObject)) return false;
-    CompositeHiveObject choOther = (CompositeHiveObject) other;
-    return this.hol.equals(choOther.hol);
-  }
-
-  @Override
-  public int hashCode() {
-    int ret = 12345;
-    for(HiveObject ho: this.hol) {
-      ret = ret * 31 + ho.hashCode();
-    }
-    return ret;
-  }
-}

+ 74 - 29
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import java.io.DataOutput;
 import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Iterator;
@@ -27,7 +27,10 @@ import java.util.List;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -47,6 +50,9 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import com.facebook.thrift.TException;
@@ -72,8 +78,10 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
     // Create the db
     Hive db;
+    FileSystem fs;
     try {
       db = Hive.get(conf);
+      fs = FileSystem.get(conf);
 
       createTableDesc crtTbl = work.getCreateTblDesc();
       if (crtTbl != null) {
@@ -91,21 +99,19 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         if (crtTbl.getNumBuckets() != -1)
           tblStorDesc.setNumBuckets(crtTbl.getNumBuckets());
         if (crtTbl.getFieldDelim() != null)
-          tbl.setFieldDelim(crtTbl.getFieldDelim());
-
+          tbl.setSerdeParam(Constants.FIELD_DELIM, crtTbl.getFieldDelim());
         if (crtTbl.getCollItemDelim() != null)
-          tbl.setCollectionItemDelim(crtTbl.getCollItemDelim());
+          tbl.setSerdeParam(Constants.COLLECTION_DELIM, crtTbl.getCollItemDelim());
         if (crtTbl.getMapKeyDelim() != null)
-          tbl.setMapKeyDelim(crtTbl.getMapKeyDelim());
+          tbl.setSerdeParam(Constants.MAPKEY_DELIM, crtTbl.getMapKeyDelim());
         if (crtTbl.getLineDelim() != null)
-          tbl.setLineDelim(crtTbl.getLineDelim());
+          tbl.setSerdeParam(Constants.LINE_DELIM, crtTbl.getLineDelim());
         if (crtTbl.getComment() != null)
           tbl.setProperty("comment", crtTbl.getComment());
         if (crtTbl.getLocation() != null)
           tblStorDesc.setLocation(crtTbl.getLocation());
 
-        tbl.setIsCompressed(crtTbl.isCompressed());
-        if (crtTbl.isCompressed()) {
+        if (crtTbl.isSequenceFile()) {
           tbl.setInputFormatClass(SequenceFileInputFormat.class);
           tbl.setOutputFormatClass(SequenceFileOutputFormat.class);
         }
@@ -116,7 +122,42 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
         if (crtTbl.isExternal())
           tbl.setProperty("EXTERNAL", "TRUE");
-        
+
+        // If the sorted columns is a superset of bucketed columns, store this fact. It can be later used to
+        // optimize some group-by queries. Note that, the order does not matter as long as it in the first
+        // 'n' columns where 'n' is the length of the bucketed columns.
+        if ((tbl.getBucketCols() != null) && (tbl.getSortCols() != null))
+        {
+          List<String> bucketCols = tbl.getBucketCols();
+          List<Order> sortCols = tbl.getSortCols();
+
+          if (sortCols.size() >= bucketCols.size())
+          {
+            boolean found = true;
+
+            Iterator<String> iterBucketCols = bucketCols.iterator();
+            while (iterBucketCols.hasNext())
+            {
+              String bucketCol = iterBucketCols.next();
+              boolean colFound = false;
+              for (int i = 0; i < bucketCols.size(); i++)
+              {
+                if (bucketCol.equals(sortCols.get(i).getCol())) {
+                  colFound = true;
+                  break;
+                }
+              }
+              if (colFound == false)
+              {
+                found = false;
+                break;
+              }
+            }
+            if (found)
+              tbl.setProperty("SORTBUCKETCOLSPREFIX", "TRUE");
+          }
+        }
+
         // create the table
         db.createTable(tbl);
         return 0;
@@ -162,21 +203,24 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           LOG.info("DDLTask: got data for " +  tbl.getName());
           
           // write the results in the file
-          FileOutputStream outStream = new FileOutputStream(descTbl.getResFile());
+          DataOutput os = (DataOutput)fs.create(descTbl.getResFile());
           List<FieldSchema> cols = tbl.getCols();
           Iterator<FieldSchema> iterCols = cols.iterator();
+          boolean firstCol = true;
           while (iterCols.hasNext())
           {
+            if (!firstCol)
+              os.write(terminator);
             FieldSchema col = iterCols.next();
-            outStream.write(col.getName().getBytes());
-            outStream.write(separator);
-            outStream.write(col.getType().getBytes());
+            os.write(col.getName().getBytes("UTF-8"));
+            os.write(separator);
+            os.write(col.getType().getBytes("UTF-8"));
             if (col.getComment() != null)
             {
-              outStream.write(separator);
-              outStream.write(col.getComment().getBytes());
+              os.write(separator);
+              os.write(col.getComment().getBytes("UTF-8"));
             }
-            outStream.write(terminator);
+            firstCol = false;
           }
 
           // also return the partitioning columns
@@ -184,19 +228,20 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           Iterator<FieldSchema> iterPartCols = partCols.iterator();
           while (iterPartCols.hasNext())
           {
+            os.write(terminator);
             FieldSchema col = iterPartCols.next();
-            outStream.write(col.getName().getBytes());
-            outStream.write(separator);
-            outStream.write(col.getType().getBytes());
+            os.write(col.getName().getBytes("UTF-8"));
+            os.write(separator);
+            os.write(col.getType().getBytes("UTF-8"));
             if (col.getComment() != null)
             {
-              outStream.write(separator);
-              outStream.write(col.getComment().getBytes());
+              os.write(separator);
+              os.write(col.getComment().getBytes("UTF-8"));
             }
-            outStream.write(terminator);
           }
-          outStream.close();
           LOG.info("DDLTask: written data for " +  tbl.getName());
+          ((FSDataOutputStream)os).close();
+          
         } catch (FileNotFoundException e) {
           LOG.info("describe table: " + StringUtils.stringifyException(e));
           return 1;
@@ -212,10 +257,10 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         if (!found)
         {
           try {
-            FileOutputStream outStream = new FileOutputStream(descTbl.getResFile());
+            DataOutput outStream = (DataOutput)fs.open(descTbl.getResFile());
             String errMsg = "Table " + descTbl.getTableName() + " does not exist";
-            outStream.write(errMsg.getBytes());
-            outStream.close();
+            outStream.write(errMsg.getBytes("UTF-8"));
+            ((FSDataOutputStream)outStream).close();
           } catch (FileNotFoundException e) {
             LOG.info("describe table: " + StringUtils.stringifyException(e));
             return 1;
@@ -243,7 +288,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         
         // write the results in the file
         try {
-          FileOutputStream outStream = new FileOutputStream(showTbls.getResFile());
+          DataOutput outStream = (DataOutput)fs.create(showTbls.getResFile());
           SortedSet<String> sortedTbls = new TreeSet<String>(tbls);
           Iterator<String> iterTbls = sortedTbls.iterator();
           boolean firstCol = true;
@@ -251,10 +296,10 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           {
             if (!firstCol)
               outStream.write(separator);
-            outStream.write(iterTbls.next().getBytes());
+            outStream.write(iterTbls.next().getBytes("UTF-8"));
             firstCol = false;
           }
-          outStream.write(terminator);
+          ((FSDataOutputStream)outStream).close();
         } catch (FileNotFoundException e) {
           LOG.info("show table: " + StringUtils.stringifyException(e));
           return 1;

+ 2 - 9
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java

@@ -147,15 +147,8 @@ public class ExecDriver extends Task<mapredWork> implements Serializable {
     FileOutputFormat.setOutputPath(job, new Path(jobScratchDir));
     job.setMapperClass(ExecMapper.class);
 
-    if(!work.getNeedsTagging()) {
-      job.setMapOutputValueClass(NoTagWritableHiveObject.class);
-      job.setMapOutputKeyClass(NoTagWritableComparableHiveObject.class);    
-      job.setOutputKeyComparatorClass(NoTagHiveObjectComparator.class);
-    } else {
-      job.setMapOutputValueClass(WritableHiveObject.class);
-      job.setMapOutputKeyClass(WritableComparableHiveObject.class);    
-      job.setOutputKeyComparatorClass(HiveObjectComparator.class);
-    }
+    job.setMapOutputValueClass(Text.class);
+    job.setMapOutputKeyClass(HiveKey.class);    
 
     job.setNumReduceTasks(work.getNumReduceTasks().intValue());
     job.setReducerClass(ExecReducer.class);

+ 103 - 28
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecReducer.java

@@ -22,29 +22,43 @@ import java.io.*;
 import java.util.*;
 
 import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.io.WritableHiveObject;
-import org.apache.hadoop.hive.ql.io.WritableComparableHiveObject;
 
 
+import org.apache.hadoop.hive.ql.plan.PlanUtils;
 import org.apache.hadoop.hive.ql.plan.mapredWork;
+import org.apache.hadoop.hive.ql.plan.tableDesc;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.exec.ExecMapper.reportStats;
+import org.apache.hadoop.hive.serde2.ColumnSet;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.MetadataListStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
 
 public class ExecReducer extends MapReduceBase implements Reducer {
 
   private JobConf jc;
-  private OutputCollector oc;
-  private Operator reducer;
+  private OutputCollector<?,?> oc;
+  private Operator<?> reducer;
   private Reporter rp;
   private boolean abort = false;
   private boolean isTagged = false;
-  private final HiveObject [] tagObjects =  new HiveObject [Byte.MAX_VALUE];
 
   private static String [] fieldNames;
   public static final Log l4j = LogFactory.getLog("ExecReducer");
 
+  // TODO: move to DynamicSerDe when it's ready
+  private Deserializer inputKeyDeserializer;
+  // Input value serde needs to be an array to support different SerDe 
+  // for different tags
+  private SerDe[] inputValueDeserializer = new SerDe[Byte.MAX_VALUE];
   static {
     ArrayList<String> fieldNameArray =  new ArrayList<String> ();
     for(Utilities.ReduceField r: Utilities.ReduceField.values()) {
@@ -53,20 +67,37 @@ public class ExecReducer extends MapReduceBase implements Reducer {
     fieldNames = fieldNameArray.toArray(new String [0]);
   }
 
-
   public void configure(JobConf job) {
     jc = job;
     mapredWork gWork = Utilities.getMapRedWork(job);
     reducer = gWork.getReducer();
     reducer.setMapredWork(gWork);
     isTagged = gWork.getNeedsTagging();
-
-    // create a hive object to encapsulate each one of the potential tags      
-    for(int i=0; i<Byte.MAX_VALUE; i++) {
-      tagObjects[i] = new PrimitiveHiveObject(Byte.valueOf((byte)i));
-    }
+    try {
+      // We should initialize the SerDe with the TypeInfo when available.
+      tableDesc keyTableDesc = PlanUtils.getReduceKeyDesc(gWork);
+      inputKeyDeserializer = (SerDe)ReflectionUtils.newInstance(keyTableDesc.getDeserializerClass(), null);
+      inputKeyDeserializer.initialize(null, keyTableDesc.getProperties());
+      for(int tag=0; tag<Byte.MAX_VALUE; tag++) {
+        // We should initialize the SerDe with the TypeInfo when available.
+        tableDesc valueTableDesc = PlanUtils.getReduceValueDesc(gWork, tag);
+        inputValueDeserializer[tag] = (SerDe)ReflectionUtils.newInstance(valueTableDesc.getDeserializerClass(), null);
+        inputValueDeserializer[tag].initialize(null, valueTableDesc.getProperties());
+      }
+    } catch (SerDeException e) {
+      throw new RuntimeException(e);
+    }    
   }
 
+  private Object keyObject;
+  private ObjectInspector keyObjectInspector;
+  private Object[] valueObject = new Object[Byte.MAX_VALUE];
+  private ObjectInspector[] valueObjectInspector = new ObjectInspector[Byte.MAX_VALUE];
+  private ObjectInspector[] rowObjectInspector = new ObjectInspector[Byte.MAX_VALUE];
+  
+  private BytesWritable groupKey;
+  
+  ArrayList<Object> row = new ArrayList<Object>(3);
   public void reduce(Object key, Iterator values,
                      OutputCollector output,
                      Reporter reporter) throws IOException {
@@ -85,26 +116,65 @@ public class ExecReducer extends MapReduceBase implements Reducer {
     }
 
     try {
-      // the key is either a WritableComparable or a NoTagWritableComparable
-      HiveObject keyObject = ((WritableComparableHiveObject)key).getHo();
-      //System.err.print(keyObject.toString());
-      // If a operator wants to do some work at the beginning of a group
-      reducer.startGroup();
-      while(values.hasNext()) {
-        WritableHiveObject who = (WritableHiveObject)values.next();
-       //System.err.print(who.getHo().toString());
-
-        LabeledCompositeHiveObject lho = new LabeledCompositeHiveObject(fieldNames);
-        lho.addHiveObject(keyObject);
-        lho.addHiveObject(who.getHo());
-        if(isTagged) {
-          lho.addHiveObject(tagObjects[who.getTag()]);
+      BytesWritable keyWritable = (BytesWritable)key;
+      byte tag = 0;
+      if (isTagged) {
+        // remove the tag
+        int size = keyWritable.getSize() - 1;
+        tag = keyWritable.get()[size]; 
+        keyWritable.setSize(size);
+      }
+      
+      if (!keyWritable.equals(groupKey)) {
+        // If a operator wants to do some work at the beginning of a group
+        if (groupKey == null) {
+          groupKey = new BytesWritable();
+        } else {
+          // If a operator wants to do some work at the end of a group
+          l4j.trace("End Group");
+          reducer.endGroup();
         }
-        reducer.process(lho);
+        groupKey.set(keyWritable.get(), 0, keyWritable.getSize());
+        l4j.trace("Start Group");
+        reducer.startGroup();
+      }
+      try {
+        keyObject = inputKeyDeserializer.deserialize(keyWritable);
+      } catch (SerDeException e) {
+        throw new HiveException(e);
+      }
+      // This is a hack for generating the correct ObjectInspector.
+      // In the future, we should use DynamicSerde and initialize it using the type info. 
+      if (keyObjectInspector == null) {
+        // Directly create ObjectInspector here because we didn't know the number of cols till now.
+        keyObjectInspector = MetadataListStructObjectInspector.getInstance(((ColumnSet)keyObject).col.size()); 
+      }
+      // System.err.print(keyObject.toString());
+      while (values.hasNext()) {
+        Text valueText = (Text)values.next();
+        //System.err.print(who.getHo().toString());
+        try {
+          valueObject[tag] = inputValueDeserializer[tag].deserialize(valueText);
+        } catch (SerDeException e) {
+          throw new HiveException(e);
+        }
+        row.clear();
+        row.add(keyObject);
+        row.add(valueObject[tag]);
+        row.add(tag);
+        if (valueObjectInspector[tag] == null) {
+          // Directly create ObjectInspector here because we didn't know the number of cols till now.
+          valueObjectInspector[tag] = MetadataListStructObjectInspector.getInstance(((ColumnSet)valueObject[tag]).col.size());
+          ArrayList<ObjectInspector> ois = new ArrayList<ObjectInspector>();
+          ois.add(keyObjectInspector);
+          ois.add(valueObjectInspector[tag]);
+          ois.add(ObjectInspectorFactory.getStandardPrimitiveObjectInspector(Byte.class));
+          rowObjectInspector[tag] = ObjectInspectorFactory.getStandardStructObjectInspector(
+              Arrays.asList(fieldNames), ois);
+        }
+        reducer.process(row, rowObjectInspector[tag]);
       }
 
-      // If a operator wants to do some work at the end of a group
-      reducer.endGroup();
 
     } catch (HiveException e) {
       abort = true;
@@ -114,6 +184,11 @@ public class ExecReducer extends MapReduceBase implements Reducer {
 
   public void close() {
     try {
+      if (groupKey != null) {
+        // If a operator wants to do some work at the end of a group
+        l4j.trace("End Group");
+        reducer.endGroup();
+      }
       reducer.close(abort);
       reportStats rps = new reportStats (rp);
       reducer.preorderMap(rps);

+ 7 - 4
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import java.io.FileOutputStream;
+import java.io.OutputStream;
 import java.io.PrintStream;
 import java.io.Serializable;
 import java.lang.annotation.Annotation;
@@ -28,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hive.ql.plan.explain;
 import org.apache.hadoop.hive.ql.plan.explainWork;
 import org.apache.hadoop.util.StringUtils;
@@ -43,9 +45,9 @@ public class ExplainTask extends Task<explainWork> implements Serializable {
   public int execute() {
     
     try {
-      // If this is an explain plan then return from here
-      PrintStream out = new PrintStream(new FileOutputStream(work.getResFile()));
-
+    	OutputStream outS = FileSystem.get(conf).create(work.getResFile());
+    	PrintStream out = new PrintStream(outS);
+    	
       // Print out the parse AST
       outputAST(work.getAstStringTree(), out, 0);
       out.println();
@@ -55,7 +57,8 @@ public class ExplainTask extends Task<explainWork> implements Serializable {
       
       // Go over all the tasks and dump out the plans
       outputStagePlans(out, work.getRootTasks(), 0);
-
+      out.close();
+      
       return (0);
     }
     catch (Exception e) {

+ 42 - 8
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeColumnEvaluator.java

@@ -20,25 +20,59 @@ package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.exprNodeColumnDesc;
-import org.apache.hadoop.hive.serde.SerDeField;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 
+/**
+ * This class support multi-level fields like "a.b.c" for historical reasons.
+ */
 public class ExprNodeColumnEvaluator extends ExprNodeEvaluator {
 
   protected exprNodeColumnDesc expr;
-  transient SerDeField field;  
+  transient StructObjectInspector cachedRowInspector;
+  transient String[] fieldNames;
+  transient StructField[] fields;
+  transient ObjectInspector[] fieldsObjectInspector;
   
   public ExprNodeColumnEvaluator(exprNodeColumnDesc expr) {
     this.expr = expr;
   }
 
-  public Object evaluateToObject(HiveObject row)  throws HiveException {
-    return evaluate(row).getJavaObject();
+  public void evaluate(Object row, ObjectInspector rowInspector,
+      InspectableObject result) throws HiveException {
+    
+    assert(result != null);
+    // If this is the first row, or the dynamic structure of this row 
+    // is different from the previous row 
+    if (fields == null || cachedRowInspector != rowInspector) {
+      evaluateInspector(rowInspector);
+    }
+    result.o = cachedRowInspector.getStructFieldData(row, fields[0]);
+    for(int i=1; i<fields.length; i++) {
+      result.o = ((StructObjectInspector)fieldsObjectInspector[i-1]).getStructFieldData(
+          result.o, fields[i]);
+    }
+    result.oi = fieldsObjectInspector[fieldsObjectInspector.length - 1];
   }
 
-  public HiveObject evaluate(HiveObject row) throws HiveException {
-    if (field == null) {
-      field = row.getFieldFromExpression(expr.getColumn());
+  public ObjectInspector evaluateInspector(ObjectInspector rowInspector)
+      throws HiveException {
+    
+    if (fields == null || cachedRowInspector != rowInspector) {
+      cachedRowInspector = (StructObjectInspector)rowInspector;
+      fieldNames = expr.getColumn().split("\\.", -1);
+      fields = new StructField[fieldNames.length];
+      fieldsObjectInspector = new ObjectInspector[fieldNames.length];
+      fields[0] = cachedRowInspector.getStructFieldRef(fieldNames[0]);
+      fieldsObjectInspector[0] = fields[0].getFieldObjectInspector();
+      for (int i=1; i<fieldNames.length; i++) {
+        fields[i] = ((StructObjectInspector)fieldsObjectInspector[i-1]).getStructFieldRef(fieldNames[i]);
+        fieldsObjectInspector[i] = fields[i].getFieldObjectInspector();
+      }
     }
-    return row.get(field);
+    return fieldsObjectInspector[fieldsObjectInspector.length - 1];
   }
 }

+ 15 - 5
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantEvaluator.java

@@ -20,20 +20,30 @@ package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.exprNodeConstantDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 
 public class ExprNodeConstantEvaluator extends ExprNodeEvaluator {
 
   protected exprNodeConstantDesc expr;
-
+  transient ObjectInspector objectInspector;
+  
   public ExprNodeConstantEvaluator(exprNodeConstantDesc expr) {
     this.expr = expr;
+    objectInspector = ObjectInspectorFactory.getStandardPrimitiveObjectInspector(expr.getTypeInfo().getPrimitiveClass());
   }
 
-  public Object evaluateToObject(HiveObject row)  throws HiveException {
-    return expr.getValue();
+  public void evaluate(Object row, ObjectInspector rowInspector,
+      InspectableObject result) throws HiveException {
+    assert(result != null);
+    result.o = expr.getValue();
+    result.oi = objectInspector;
   }
 
-  public HiveObject evaluate(HiveObject r) throws HiveException {
-    return new PrimitiveHiveObject(evaluateToObject(r));
+  public ObjectInspector evaluateInspector(ObjectInspector rowInspector)
+      throws HiveException {
+    return objectInspector;
   }
 }

+ 12 - 7
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluator.java

@@ -19,16 +19,21 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 public abstract class ExprNodeEvaluator {
+
   /**
-   * @return plain old java object
-   **/
-  public abstract Object evaluateToObject(HiveObject row) throws HiveException;
+   * Evaluate the expression given the row and rowInspector. 
+   * @param result   result.o and result.oi will be set inside the method.
+   */
+  public abstract void evaluate(Object row, ObjectInspector rowInspector, InspectableObject result) throws HiveException;
 
   /**
-   * @return encapsulated Hive Object
-   **/
-  public abstract HiveObject evaluate(HiveObject row) throws HiveException;
-  
+   * Metadata evaluation. Return the inspector for the expression, given the rowInspector.
+   * This method must return the same value as result.oi in evaluate(...) call with the same rowInspector.   
+   */
+  public abstract ObjectInspector evaluateInspector(ObjectInspector rowInspector) throws HiveException;
+
 }

+ 40 - 10
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeFieldEvaluator.java

@@ -20,29 +20,59 @@ package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.exprNodeFieldDesc;
-import org.apache.hadoop.hive.serde.SerDeField;
+
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 
 public class ExprNodeFieldEvaluator extends ExprNodeEvaluator {
 
   protected exprNodeFieldDesc desc;
-  transient ExprNodeEvaluator evaluator;
-  transient SerDeField field;
+  transient ExprNodeEvaluator leftEvaluator;
+  transient InspectableObject leftInspectableObject;
+  transient StructObjectInspector cachedLeftObjectInspector;
+  transient StructField field;
+  transient ObjectInspector fieldObjectInspector;
   
   public ExprNodeFieldEvaluator(exprNodeFieldDesc desc) {
     this.desc = desc;
-    evaluator = ExprNodeEvaluatorFactory.get(desc.getDesc());
+    leftEvaluator = ExprNodeEvaluatorFactory.get(desc.getDesc());
+    field = null;
+    leftInspectableObject = new InspectableObject();
   }
 
+  public void evaluate(Object row, ObjectInspector rowInspector,
+      InspectableObject result) throws HiveException {
+    
+    assert(result != null);
+    // Get the result in leftInspectableObject
+    leftEvaluator.evaluate(row, rowInspector, leftInspectableObject);
 
-  public Object evaluateToObject(HiveObject row)  throws HiveException {
-    return evaluate(row).getJavaObject();
+    if (field == null) {
+      cachedLeftObjectInspector = (StructObjectInspector)leftInspectableObject.oi;
+      field = cachedLeftObjectInspector.getStructFieldRef(desc.getFieldName());
+      fieldObjectInspector = field.getFieldObjectInspector();
+    } else {
+      assert(cachedLeftObjectInspector == leftInspectableObject.oi);
+    }
+    result.oi = fieldObjectInspector;
+    result.o = cachedLeftObjectInspector.getStructFieldData(leftInspectableObject.o, field); 
   }
 
-  public HiveObject evaluate(HiveObject row) throws HiveException {
-    HiveObject ho = evaluator.evaluate(row);
+  public ObjectInspector evaluateInspector(ObjectInspector rowInspector)
+      throws HiveException {
+    // If this is the first row, or the dynamic structure of the evaluatorInspectableObject 
+    // is different from the previous row 
+    leftInspectableObject.oi = leftEvaluator.evaluateInspector(rowInspector);
     if (field == null) {
-      field = ho.getFieldFromExpression(desc.getFieldName());
+      cachedLeftObjectInspector = (StructObjectInspector)leftInspectableObject.oi;
+      field = cachedLeftObjectInspector.getStructFieldRef(desc.getFieldName());
+      fieldObjectInspector = field.getFieldObjectInspector();
+    } else {
+      assert(cachedLeftObjectInspector == leftInspectableObject.oi);      
     }
-    return ho.get(field);
+    return fieldObjectInspector;
   }
+
 }

+ 42 - 20
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeFuncEvaluator.java

@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
-import java.util.ArrayList;
+import java.lang.reflect.Method;
 import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.exprNodeFuncDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.util.ReflectionUtils;
 
 public class ExprNodeFuncEvaluator extends ExprNodeEvaluator {
@@ -32,41 +35,60 @@ public class ExprNodeFuncEvaluator extends ExprNodeEvaluator {
   private static final Log LOG = LogFactory.getLog(ExprNodeFuncEvaluator.class.getName());
   
   protected exprNodeFuncDesc expr;
-  transient ArrayList<ExprNodeEvaluator> evaluators;
-  transient Object[] children;
+  transient ExprNodeEvaluator[] paramEvaluators;
+  transient InspectableObject[] paramInspectableObjects;
+  transient Object[] paramValues;
   transient UDF udf;
+  transient Method udfMethod;
+  transient ObjectInspector outputObjectInspector;
   
   public ExprNodeFuncEvaluator(exprNodeFuncDesc expr) {
     this.expr = expr;
     assert(expr != null);
     Class<?> c = expr.getUDFClass();
-    LOG.info(c.toString());
+    udfMethod = expr.getUDFMethod();
+    LOG.debug(c.toString());
+    LOG.debug(udfMethod.toString());
     udf = (UDF)ReflectionUtils.newInstance(expr.getUDFClass(), null);
-    evaluators = new ArrayList<ExprNodeEvaluator>();
-    for(int i=0; i<expr.getChildren().size(); i++) {
-      evaluators.add(ExprNodeEvaluatorFactory.get(expr.getChildren().get(i)));
+    int paramNumber = expr.getChildren().size();
+    paramEvaluators = new ExprNodeEvaluator[paramNumber];
+    paramInspectableObjects  = new InspectableObject[paramNumber];
+    for(int i=0; i<paramNumber; i++) {
+      paramEvaluators[i] = ExprNodeEvaluatorFactory.get(expr.getChildren().get(i));
+      paramInspectableObjects[i] = new InspectableObject();
     }
-    children = new Object[expr.getChildren().size()];  
+    paramValues = new Object[expr.getChildren().size()];
+    outputObjectInspector = ObjectInspectorFactory.getStandardPrimitiveObjectInspector(
+        udfMethod.getReturnType());
   }
 
-  public Object evaluateToObject(HiveObject row)  throws HiveException {
+  public void evaluate(Object row, ObjectInspector rowInspector,
+      InspectableObject result) throws HiveException {
+    if (result == null) {
+      throw new HiveException("result cannot be null.");
+    }
     // Evaluate all children first
-    for(int i=0; i<evaluators.size(); i++) {
-      Object o = evaluators.get(i).evaluateToObject(row);
-      children[i] = o;
+    for(int i=0; i<paramEvaluators.length; i++) {
+      paramEvaluators[i].evaluate(row, rowInspector, paramInspectableObjects[i]);
+      paramValues[i] = paramInspectableObjects[i].o;
     }
     try {
-      return expr.getUDFMethod().invoke(udf, children);
+      result.o = udfMethod.invoke(udf, paramValues);
+      result.oi = outputObjectInspector;
     } catch (Exception e) {
-      throw new HiveException("Unable to execute UDF function " + udf.getClass() + " " 
-          + expr.getUDFMethod() + " on inputs " + "(" + children.length + ") " + Arrays.asList(children) + ": " + e.getMessage(), e);
+      if (e instanceof HiveException) {
+        throw (HiveException)e;
+      } else if (e instanceof RuntimeException) {
+        throw (RuntimeException)e;
+      } else {
+        throw new HiveException("Unable to execute UDF function " + udf.getClass() + " " 
+          + udfMethod + " on inputs " + "(" + paramValues.length + ") " + Arrays.asList(paramValues) + ": " + e.getMessage(), e);
+      }
     }
   }
 
-  public HiveObject evaluate(HiveObject row) throws HiveException {
-    Object obj = evaluateToObject(row);
-    if (obj == null)
-      return new NullHiveObject();
-    return new PrimitiveHiveObject(obj);
+  public ObjectInspector evaluateInspector(ObjectInspector rowInspector)
+      throws HiveException {
+    return outputObjectInspector;
   }
 }

+ 20 - 13
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeIndexEvaluator.java

@@ -20,14 +20,17 @@ package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.exprNodeIndexDesc;
-import org.apache.hadoop.hive.serde.SerDeField;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 public class ExprNodeIndexEvaluator extends ExprNodeEvaluator {
 
   protected exprNodeIndexDesc expr;
   transient ExprNodeEvaluator mainEvaluator;
+  transient InspectableObject mainInspectableObject = new InspectableObject();
   transient ExprNodeEvaluator indexEvaluator;
-  transient SerDeField field;
+  transient InspectableObject indexInspectableObject = new InspectableObject();
   
   public ExprNodeIndexEvaluator(exprNodeIndexDesc expr) {
     this.expr = expr;
@@ -35,18 +38,22 @@ public class ExprNodeIndexEvaluator extends ExprNodeEvaluator {
     indexEvaluator = ExprNodeEvaluatorFactory.get(expr.getIndex());
   }
 
-  public Object evaluateToObject(HiveObject row)  throws HiveException {
-    return evaluate(row).getJavaObject();
+  public void evaluate(Object row, ObjectInspector rowInspector,
+      InspectableObject result) throws HiveException {
+    
+    assert(result != null);
+    mainEvaluator.evaluate(row, rowInspector, mainInspectableObject);
+    indexEvaluator.evaluate(row, rowInspector, indexInspectableObject);
+    int index = ((Number)indexInspectableObject.o).intValue();
+    
+    ListObjectInspector loi = (ListObjectInspector)mainInspectableObject.oi;
+    result.oi = loi.getListElementObjectInspector();
+    result.o = loi.getListElement(mainInspectableObject.o, index);
   }
 
-  public HiveObject evaluate(HiveObject row) throws HiveException {
-    HiveObject ho = mainEvaluator.evaluate(row);
-    if (field == null || !(indexEvaluator instanceof ExprNodeConstantEvaluator)) {
-      // TODO: This optimization is wrong because of the field implementation inside HiveObject.
-      // The problem is that at the second "[" (after "c"), "field" caches both "index1" and 
-      // "index2" in "a.b[index1].c[index2]", while it should only cache "index2".
-      field = ho.getFieldFromExpression("[" + indexEvaluator.evaluateToObject(row) + "]");
-    }
-    return ho.get(field);
+  public ObjectInspector evaluateInspector(ObjectInspector rowInspector)
+      throws HiveException {
+    return ((ListObjectInspector)mainEvaluator.evaluateInspector(rowInspector)).getListElementObjectInspector();
   }
+
 }

+ 9 - 4
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeNullEvaluator.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.exprNodeNullDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 // This function will not be used currently, since the function expressions
 // change the void to the first matching argument
@@ -31,11 +33,14 @@ public class ExprNodeNullEvaluator extends ExprNodeEvaluator {
     this.expr = expr;
   }
 
-  public Object evaluateToObject(HiveObject row)  throws HiveException {
-    return expr.getValue();
+  public void evaluate(Object row, ObjectInspector rowInspector,
+      InspectableObject result) throws HiveException {
+    throw new HiveException("Hive 2 Internal exception: should not reach here.");
   }
 
-  public HiveObject evaluate(HiveObject r) throws HiveException {
-    return new NullHiveObject();
+  public ObjectInspector evaluateInspector(ObjectInspector rowInspector)
+      throws HiveException {
+    throw new HiveException("Hive 2 Internal exception: should not reach here.");
   }
+
 }

+ 6 - 2
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExtractOperator.java

@@ -22,6 +22,8 @@ import java.io.*;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.extractDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -31,13 +33,15 @@ import org.apache.hadoop.conf.Configuration;
 public class ExtractOperator extends Operator<extractDesc> implements Serializable {
   private static final long serialVersionUID = 1L;
   transient protected ExprNodeEvaluator eval;
+  transient protected InspectableObject result = new InspectableObject();
 
   public void initialize(Configuration hconf) throws HiveException {
     super.initialize(hconf);
     eval = ExprNodeEvaluatorFactory.get(conf.getCol());
   }
 
-  public void process(HiveObject r) throws HiveException {
-    forward (eval.evaluate(r));
+  public void process(Object row, ObjectInspector rowInspector) throws HiveException {
+    eval.evaluate(row, rowInspector, result);
+    forward(result.o, result.oi);
   }
 }

+ 32 - 14
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java

@@ -23,16 +23,14 @@ import java.io.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.fs.*;
-import org.apache.hadoop.util.*;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.fileSinkDesc;
 import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
-import org.apache.hadoop.hive.serde.SerDe;
-import org.apache.hadoop.hive.serde.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.Serializer;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 /**
  * File Sink operator implementation
@@ -49,7 +47,7 @@ public class FileSinkOperator extends TerminalOperator <fileSinkDesc> implements
   transient protected FileSystem fs;
   transient protected Path outPath;
   transient protected Path finalPath;
-  transient protected SerDe serDe;
+  transient protected Serializer serializer;
   transient protected BytesWritable commonKey = new BytesWritable();
   
   private void commit() throws IOException {
@@ -79,13 +77,31 @@ public class FileSinkOperator extends TerminalOperator <fileSinkDesc> implements
   public void initialize(Configuration hconf) throws HiveException {
     super.initialize(hconf);
     try {
+      serializer = (Serializer)conf.getTableInfo().getDeserializerClass().newInstance();
+      serializer.initialize(null, conf.getTableInfo().getProperties());
+      
+      JobConf jc;
+      if(hconf instanceof JobConf) {
+        jc = (JobConf)hconf;
+      } else {
+        // test code path
+        jc = new JobConf(hconf, ExecDriver.class);
+      }
+
       fs = FileSystem.get(hconf);
       finalPath = new Path(conf.getDirName(), Utilities.getTaskId(hconf));
       outPath = new Path(conf.getDirName(), "_tmp."+Utilities.getTaskId(hconf));
-      OutputFormat outputFormat = conf.getTableInfo().getOutputFileFormatClass().newInstance();
+      OutputFormat<?, ?> outputFormat = conf.getTableInfo().getOutputFileFormatClass().newInstance();
+      final Class<? extends Writable> outputClass = serializer.getSerializedClass();
+      boolean isCompressed = FileOutputFormat.getCompressOutput(jc);
 
+      // The reason to keep these instead of using OutputFormat.getRecordWriter() is that
+      // getRecordWriter does not give us enough control over the file name that we create.
       if(outputFormat instanceof IgnoreKeyTextOutputFormat) {
-        final FSDataOutputStream outStream = fs.create(outPath);
+        if(isCompressed) {
+          finalPath = new Path(conf.getDirName(), Utilities.getTaskId(hconf) + ".gz");
+        }
+        final OutputStream outStream = Utilities.createCompressedStream(jc, fs.create(outPath));
         outWriter = new RecordWriter () {
             public void write(Writable r) throws IOException {
               Text tr = (Text)r;
@@ -98,7 +114,7 @@ public class FileSinkOperator extends TerminalOperator <fileSinkDesc> implements
           };
       } else if (outputFormat instanceof SequenceFileOutputFormat) {
         final SequenceFile.Writer outStream =
-          SequenceFile.createWriter(fs, hconf, outPath, BytesWritable.class, Text.class);
+            Utilities.createSequenceWriter(jc, fs, outPath, BytesWritable.class, outputClass);
         outWriter = new RecordWriter () {
             public void write(Writable r) throws IOException {
               outStream.append(commonKey, r);
@@ -109,20 +125,22 @@ public class FileSinkOperator extends TerminalOperator <fileSinkDesc> implements
           };
       } else {
         // should never come here - we should be catching this in ddl command
-        assert(false);
+        throw new HiveException ("Illegal outputformat: " + outputFormat.getClass().getName());
       }
-      serDe = conf.getTableInfo().getSerdeClass().newInstance();
+    } catch (HiveException e) {
+      throw e;
     } catch (Exception e) {
       e.printStackTrace();
       throw new HiveException(e);
     }
   }
 
-  public void process(HiveObject r) throws HiveException {
+  Writable recordValue; 
+  public void process(Object row, ObjectInspector rowInspector) throws HiveException {
     try {
       // user SerDe to serialize r, and write it out
-      Writable value = serDe.serialize(r.getJavaObject());
-      outWriter.write(value);
+      recordValue = serializer.serialize(row, rowInspector);
+      outWriter.write(recordValue);
     } catch (IOException e) {
       throw new HiveException (e);
     } catch (SerDeException e) {

+ 12 - 9
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java

@@ -23,6 +23,8 @@ import java.io.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.hive.ql.plan.filterDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -33,18 +35,20 @@ public class FilterOperator extends Operator <filterDesc> implements Serializabl
   private static final long serialVersionUID = 1L;
   public static enum Counter {FILTERED, PASSED}
   transient private final LongWritable filtered_count, passed_count;
-  transient private ExprNodeEvaluator eval;
-
+  transient private ExprNodeEvaluator conditionEvaluator;
+  transient private InspectableObject conditionInspectableObject;  
+  
   public FilterOperator () {
     super();
     filtered_count = new LongWritable();
     passed_count = new LongWritable();
+    conditionInspectableObject = new InspectableObject();
   }
 
   public void initialize(Configuration hconf) throws HiveException {
     super.initialize(hconf);
     try {
-      this.eval = ExprNodeEvaluatorFactory.get(conf.getPredicate());
+      this.conditionEvaluator = ExprNodeEvaluatorFactory.get(conf.getPredicate());
       statsMap.put(Counter.FILTERED, filtered_count);
       statsMap.put(Counter.PASSED, passed_count);
     } catch (Throwable e) {
@@ -53,11 +57,12 @@ public class FilterOperator extends Operator <filterDesc> implements Serializabl
     }
   }
 
-  public void process(HiveObject r) throws HiveException {
+  public void process(Object row, ObjectInspector rowInspector) throws HiveException {
     try {
-      Boolean ret = (Boolean)(eval.evaluateToObject(r));
+      conditionEvaluator.evaluate(row, rowInspector, conditionInspectableObject);
+      Boolean ret = (Boolean)(conditionInspectableObject.o);
       if (Boolean.TRUE.equals(ret)) {
-        forward(r);
+        forward(row, rowInspector);
         passed_count.set(passed_count.get()+1);
       } else {
         filtered_count.set(filtered_count.get()+1);
@@ -65,9 +70,7 @@ public class FilterOperator extends Operator <filterDesc> implements Serializabl
     } catch (ClassCastException e) {
       e.printStackTrace();
       throw new HiveException("Non Boolean return Object type: " +
-                              eval.evaluateToObject(r).getClass().getName());
-    } catch (NullPointerException e) {
-      throw new HiveException("NullPointerException in FilterOperator ", e);
+          conditionInspectableObject.o.getClass().getName());
     }
   }
 }

+ 5 - 2
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ForwardOperator.java

@@ -22,6 +22,7 @@ import java.io.*;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.forwardDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -35,7 +36,9 @@ public class ForwardOperator extends  Operator<forwardDesc>  implements Serializ
     // nothing to do really ..
   }
 
-  public void process(HiveObject r) throws HiveException {
-    forward(r);
+  @Override
+  public void process(Object row, ObjectInspector rowInspector)
+      throws HiveException {
+    forward(row, rowInspector);    
   }
 }

+ 2 - 2
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java

@@ -29,8 +29,8 @@ import java.util.Map;
 import java.lang.Void;
 
 import org.apache.hadoop.hive.ql.exec.FunctionInfo.OperatorType;
-import org.apache.hadoop.hive.ql.parse.TypeInfo;
 import org.apache.hadoop.hive.ql.udf.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 
 public class FunctionRegistry {
 
@@ -252,7 +252,7 @@ public class FunctionRegistry {
 
         for(int i=0; i<argumentClasses.size() && match; i++) {
           if (argumentClasses.get(i) == Void.class) continue;
-          Class<?> accepted = TypeInfo.generalizePrimitive(argumentTypeInfos[i]);
+          Class<?> accepted = ObjectInspectorUtils.generalizePrimitive(argumentTypeInfos[i]);
           if (accepted.isAssignableFrom(argumentClasses.get(i))) {
             // do nothing if match
           } else if (!exact && implicitConvertable(argumentClasses.get(i), accepted)) {

+ 67 - 41
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java

@@ -28,7 +28,9 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.aggregationDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.groupByDesc;
-import org.apache.hadoop.hive.serde.SerDeField;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -47,15 +49,18 @@ public class GroupByOperator extends Operator <groupByDesc> implements Serializa
   transient protected Method[] aggregationsAggregateMethods;
   transient protected Method[] aggregationsEvaluateMethods;
 
-  transient protected List<SerDeField> choKeyFields;
+  transient protected ArrayList<ObjectInspector> objectInspectors;
+  transient protected ObjectInspector outputObjectInspector;
 
   // Used by sort-based GroupBy: Mode = COMPLETE, PARTIAL1, PARTIAL2
-  transient protected CompositeHiveObject currentKeys;
+  transient protected ArrayList<Object> currentKeys;
   transient protected UDAF[] aggregations;
   transient protected Object[][] aggregationsParametersLastInvoke;
 
   // Used by hash-based GroupBy: Mode = HASH
-  transient protected HashMap<CompositeHiveObject, UDAF[]> hashAggregations;
+  transient protected HashMap<ArrayList<Object>, UDAF[]> hashAggregations;
+  
+  transient boolean firstRow;
   
   public void initialize(Configuration hconf) throws HiveException {
     super.initialize(hconf);
@@ -124,8 +129,20 @@ public class GroupByOperator extends Operator <groupByDesc> implements Serializa
         aggregationsParametersLastInvoke = new Object[conf.getAggregators().size()][];
         aggregations = newAggregations();
       } else {
-        hashAggregations = new HashMap<CompositeHiveObject, UDAF[]>();
+        hashAggregations = new HashMap<ArrayList<Object>, UDAF[]>();
+      }
+      // init objectInspectors
+      int totalFields = keyFields.length + aggregationClasses.length;
+      objectInspectors = new ArrayList<ObjectInspector>(totalFields);
+      for(int i=0; i<keyFields.length; i++) {
+        objectInspectors.add(null);
       }
+      for(int i=0; i<aggregationClasses.length; i++) {
+        objectInspectors.add(ObjectInspectorFactory.getStandardPrimitiveObjectInspector(
+            aggregationsEvaluateMethods[i].getReturnType()));
+      }
+      
+      firstRow = true;
     } catch (Exception e) {
       e.printStackTrace();
       throw new RuntimeException(e);
@@ -141,12 +158,15 @@ public class GroupByOperator extends Operator <groupByDesc> implements Serializa
     return aggs;
   }
 
-  protected void updateAggregations(UDAF[] aggs, HiveObject row, Object[][] lastInvoke) throws Exception {
+  InspectableObject tempInspectableObject = new InspectableObject();
+  
+  protected void updateAggregations(UDAF[] aggs, Object row, ObjectInspector rowInspector, Object[][] lastInvoke) throws Exception {
     for(int ai=0; ai<aggs.length; ai++) {
       // Calculate the parameters 
       Object[] o = new Object[aggregationParameterFields[ai].length];
       for(int pi=0; pi<aggregationParameterFields[ai].length; pi++) {
-        o[pi] = aggregationParameterFields[ai][pi].evaluateToObject(row);
+        aggregationParameterFields[ai][pi].evaluate(row, rowInspector, tempInspectableObject);
+        o[pi] = tempInspectableObject.o; 
       }
       // Update the aggregations.
       if (aggregationIsDistinct[ai] && lastInvoke != null) {
@@ -170,36 +190,48 @@ public class GroupByOperator extends Operator <groupByDesc> implements Serializa
     }
   }
   
-  public void process(HiveObject row) throws HiveException {
+  public void process(Object row, ObjectInspector rowInspector) throws HiveException {
     
     try {
       // Compute the keys
-      ArrayList<HiveObject> keys = new ArrayList<HiveObject>(keyFields.length);
+      ArrayList<Object> newKeys = new ArrayList<Object>(keyFields.length);
       for (int i = 0; i < keyFields.length; i++) {
-        keys.add(keyFields[i].evaluate(row));
+        keyFields[i].evaluate(row, rowInspector, tempInspectableObject);
+        newKeys.add(tempInspectableObject.o);
+        if (firstRow) {
+          objectInspectors.set(i, tempInspectableObject.oi);
+        }
+      }
+      if (firstRow) {
+        firstRow = false;
+        ArrayList<String> fieldNames = new ArrayList<String>(objectInspectors.size());
+        for(int i=0; i<objectInspectors.size(); i++) {
+          fieldNames.add(Integer.valueOf(i).toString());
+        }
+        outputObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(
+          fieldNames, objectInspectors);
       }
-      CompositeHiveObject newKeys = new CompositeHiveObject(keys); 
-      
       // Prepare aggs for updating
       UDAF[] aggs = null;
       Object[][] lastInvoke = null;
       if (aggregations != null) {
         // sort-based aggregation
         // Need to forward?
-        if (currentKeys != null && !newKeys.equals(currentKeys)) {
-            forward(currentKeys, aggregations);
+        boolean keysAreEqual = newKeys.equals(currentKeys);
+        if (currentKeys != null && !keysAreEqual) {
+          forward(currentKeys, aggregations);
         }
         // Need to update the keys?
-        if (currentKeys == null || !newKeys.equals(currentKeys)) {
-            currentKeys = newKeys;
-            // init aggregations
-            for(UDAF aggregation: aggregations) {
-                aggregation.init();
-            }
-            // clear parameters in last-invoke
-            for(int i=0; i<aggregationsParametersLastInvoke.length; i++) {
-              aggregationsParametersLastInvoke[i] = null;
-            }
+        if (currentKeys == null || !keysAreEqual) {
+          currentKeys = newKeys;
+          // init aggregations
+          for(UDAF aggregation: aggregations) {
+            aggregation.init();
+          }
+          // clear parameters in last-invoke
+          for(int i=0; i<aggregationsParametersLastInvoke.length; i++) {
+            aggregationsParametersLastInvoke[i] = null;
+          }
         }
         aggs = aggregations;
         lastInvoke = aggregationsParametersLastInvoke;
@@ -215,7 +247,7 @@ public class GroupByOperator extends Operator <groupByDesc> implements Serializa
       }
 
       // Update the aggs
-      updateAggregations(aggs, row, lastInvoke);
+      updateAggregations(aggs, row, rowInspector, lastInvoke);
 
     } catch (Exception e) {
       e.printStackTrace();
@@ -230,23 +262,16 @@ public class GroupByOperator extends Operator <groupByDesc> implements Serializa
    *          The keys in the record
    * @throws HiveException
    */
-  protected void forward(CompositeHiveObject keys, UDAF[] aggs) throws Exception {
-    if (choKeyFields == null) {
-      // init choKeyFields
-      choKeyFields = new ArrayList<SerDeField>();
-      for (int i = 0; i < keyFields.length; i++) {
-        choKeyFields.add(keys.getFieldFromExpression(Integer.valueOf(i).toString()));
-      }
+  protected void forward(ArrayList<Object> keys, UDAF[] aggs) throws Exception {
+    int totalFields = keys.size() + aggs.length;
+    List<Object> a = new ArrayList<Object>(totalFields);
+    for(int i=0; i<keys.size(); i++) {
+      a.add(keys.get(i));
     }
-    int totalFields = keys.width + aggs.length;
-    CompositeHiveObject cho = new CompositeHiveObject(totalFields);
-    for (int i = 0; i < keys.width; i++) {
-      cho.addHiveObject(keys.get(choKeyFields.get(i)));
+    for(int i=0; i<aggs.length; i++) {
+      a.add(aggregationsEvaluateMethods[i].invoke(aggs[i]));
     }
-    for (int i = 0; i < aggs.length; i++) {
-      cho.addHiveObject(new PrimitiveHiveObject(aggregationsEvaluateMethods[i].invoke(aggs[i])));
-    }
-    forward(cho);
+    forward(a, outputObjectInspector);
   }
   
   /**
@@ -263,7 +288,7 @@ public class GroupByOperator extends Operator <groupByDesc> implements Serializa
           }
         } else if (hashAggregations != null) {
           // hash-based aggregations
-          for (CompositeHiveObject key: hashAggregations.keySet()) {
+          for (ArrayList<Object> key: hashAggregations.keySet()) {
             forward(key, hashAggregations.get(key));
           }
         } else {
@@ -278,4 +303,5 @@ public class GroupByOperator extends Operator <groupByDesc> implements Serializa
     }
     super.close(abort);
   }
+
 }

+ 0 - 97
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/HiveObject.java

@@ -1,97 +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.ql.exec;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.io.*;
-import org.apache.hadoop.hive.utils.ByteStream;
-
-/**
- * Data for each row is passed around as HiveObjects in Hive
- */
-
-public abstract class HiveObject {
-
-  protected Object javaObject;
-
-  protected boolean isNull;
-
-  /**
-   * @param expr a well formed expression nesting within this Hive Object
-   * @return field handler that can be used in a subsequent get() call
-   */
-  public abstract SerDeField getFieldFromExpression(String expr) throws HiveException;
-
-  /**
-   * @param field obtained using call to getFieldFromExpression
-   * @return another subObject
-   */
-  public abstract HiveObject get(SerDeField field) throws HiveException;
-
-  /**
-   * @return get the current HiveObject as a Java Object
-   */
-  public Object getJavaObject() throws HiveException {
-    if (isNull) return null;
-    return javaObject;
-  }
-
-  /**
-   * @return get isNull
-   */
-  public boolean getIsNull() {
-    return isNull;
-  }
-
-  public void setIsNull(boolean isNull) {
-    this.isNull = isNull;
-  }
-
-  /**
-   * @return list of top level fields in this Hive Object
-   */
-  public abstract List<SerDeField> getFields() throws HiveException;
-
-  /**
-   * Used to detect base case of object hierarchy
-   * @return true if the Object encapsulates a Hive Primitive Object. False otherwise
-   */
-  public abstract boolean isPrimitive();
-
-  public abstract int hashCode();
-  
-  public abstract boolean equals(Object other);
-  
-  public String toString () {
-    try {
-      HiveObjectSerializer hos = new NaiiveSerializer();
-      ByteStream.Output bos = new ByteStream.Output ();
-      hos.serialize(this, new DataOutputStream(bos));
-      return new String(bos.getData(), 0, bos.getCount(), "UTF-8");
-    } catch (Exception e) {
-      return ("Exception:  "+e.getMessage());
-    }
-  }
-
-  public static final ArrayList<SerDeField> nlist = new ArrayList<SerDeField> (0);
-}

+ 68 - 58
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java

@@ -22,7 +22,6 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.Stack;
 import java.util.Vector;
@@ -32,8 +31,11 @@ import org.apache.hadoop.hive.ql.plan.exprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.joinCond;
 import org.apache.hadoop.hive.ql.plan.joinDesc;
-import org.apache.hadoop.hive.serde.SerDeField;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 
 /**
  * Join operator implementation.
@@ -43,35 +45,29 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
   // a list of value expressions for each alias are maintained 
   public static class JoinExprMap {
     ExprNodeEvaluator[] valueFields;
-    List<SerDeField> listFields;
 
-    public JoinExprMap(ExprNodeEvaluator[] valueFields,
-        List<SerDeField> listFields) {
+    public JoinExprMap(ExprNodeEvaluator[] valueFields) {
       this.valueFields = valueFields;
-      this.listFields = listFields;
     }
 
     public ExprNodeEvaluator[] getValueFields() {
       return valueFields;
     }
 
-    public List<SerDeField> getListFields() {
-      return listFields;
-    }
   }
 
   public static class IntermediateObject{
-    CompositeHiveObject[] objs;
+    ArrayList<Object>[] objs;
     int curSize;
 
-    public IntermediateObject(CompositeHiveObject[] objs, int curSize) {
+    public IntermediateObject(ArrayList<Object>[] objs, int curSize) {
       this.objs  = objs;
       this.curSize = curSize;
     }
 
-    public CompositeHiveObject[] getObjs() { return objs; }
+    public ArrayList<Object>[] getObjs() { return objs; }
     public int getCurSize() { return curSize; }
-    public void pushObj(CompositeHiveObject obj) { objs[curSize++] = obj; }
+    public void pushObj(ArrayList<Object> obj) { objs[curSize++] = obj; }
     public void popObj() { curSize--; }
   }
 
@@ -81,23 +77,24 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
   transient static protected Byte[] order; // order in which the results should be outputted
   transient protected joinCond[] condn;
   transient protected boolean noOuterJoin;
-  transient private HiveObject[] dummyObj; // for outer joins, contains the potential nulls for the concerned aliases
-  transient private Vector<CompositeHiveObject>[] dummyObjVectors;
-  transient private Stack<Iterator<CompositeHiveObject>> iterators;
+  transient private Object[] dummyObj; // for outer joins, contains the potential nulls for the concerned aliases
+  transient private Vector<ArrayList<Object>>[] dummyObjVectors;
+  transient private Stack<Iterator<ArrayList<Object>>> iterators;
   transient private int totalSz; // total size of the composite object
-
+  transient ObjectInspector joinOutputObjectInspector;
+  
   static
   {
     aliasField = ExprNodeEvaluatorFactory.get(new exprNodeColumnDesc(String.class, Utilities.ReduceField.ALIAS.toString()));
   }
   
-  HashMap<Byte, Vector<CompositeHiveObject>> storage;
+  HashMap<Byte, Vector<ArrayList<Object>>> storage;
 
   public void initialize(Configuration hconf) throws HiveException {
     super.initialize(hconf);
     totalSz = 0;
     // Map that contains the rows for each alias
-    storage = new HashMap<Byte, Vector<CompositeHiveObject>>();
+    storage = new HashMap<Byte, Vector<ArrayList<Object>>>();
     
     numValues = conf.getExprs().size();
     joinExprs = new HashMap<Byte, JoinExprMap>();
@@ -123,51 +120,61 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
       for (int j = 0; j < sz; j++)
         valueFields[j] = ExprNodeEvaluatorFactory.get(expr.get(j));
 
-      joinExprs.put(key, new JoinExprMap(valueFields, CompositeHiveObject
-          .getFields(sz)));
+      joinExprs.put(key, new JoinExprMap(valueFields));
     }
 
-    dummyObj = new HiveObject[numValues];
+    ArrayList<ObjectInspector> structFieldObjectInspectors = new ArrayList<ObjectInspector>(totalSz);
+    for(int i=0; i<totalSz; i++) {
+      structFieldObjectInspectors.add(ObjectInspectorFactory.getStandardPrimitiveObjectInspector(String.class));
+    }
+    joinOutputObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(
+        ObjectInspectorUtils.getIntegerArray(totalSz), structFieldObjectInspectors);
+
+    dummyObj = new Object[numValues];
     dummyObjVectors = new Vector[numValues];
 
     int pos = 0;
     for (Byte alias : order) {
       int sz = map.get(alias).size();
-      CompositeHiveObject nr = new CompositeHiveObject(sz);
+      ArrayList<Object> nr = new ArrayList<Object>(sz);
 
       for (int j = 0; j < sz; j++)
-        nr.addHiveObject(null);
+        nr.add(null);
 
       dummyObj[pos] = nr;
-      Vector<CompositeHiveObject> values = new Vector<CompositeHiveObject>();
-      values.add((CompositeHiveObject) dummyObj[pos]);
+      Vector<ArrayList<Object>> values = new Vector<ArrayList<Object>>();
+      values.add((ArrayList<Object>) dummyObj[pos]);
       dummyObjVectors[pos] = values;
       pos++;
     }
 
-    iterators = new Stack<Iterator<CompositeHiveObject>>();
+    iterators = new Stack<Iterator<ArrayList<Object>>>();
   }
 
   public void startGroup() throws HiveException {
-    l4j.trace("Join: Starting new group");
+    LOG.trace("Join: Starting new group");
     storage.clear();
     for (Byte alias : order)
-      storage.put(alias, new Vector<CompositeHiveObject>());
+      storage.put(alias, new Vector<ArrayList<Object>>());
   }
 
-  public void process(HiveObject row) throws HiveException {
+  InspectableObject tempAliasInspectableObject = new InspectableObject();
+  public void process(Object row, ObjectInspector rowInspector) throws HiveException {
     try {
       // get alias
-      Byte alias = (Byte) (aliasField.evaluate(row).getJavaObject());
+      aliasField.evaluate(row, rowInspector, tempAliasInspectableObject);
+      Byte alias = (Byte) (tempAliasInspectableObject.o);
 
       // get the expressions for that alias
       JoinExprMap exmap = joinExprs.get(alias);
       ExprNodeEvaluator[] valueFields = exmap.getValueFields();
 
       // Compute the values
-      CompositeHiveObject nr = new CompositeHiveObject(valueFields.length);
-      for (ExprNodeEvaluator vField : valueFields)
-        nr.addHiveObject(vField.evaluate(row));
+      ArrayList<Object> nr = new ArrayList<Object>(valueFields.length);
+      for (ExprNodeEvaluator vField : valueFields) {
+        vField.evaluate(row, rowInspector, tempAliasInspectableObject);
+        nr.add(tempAliasInspectableObject.o);
+      }
 
       // Add the value to the vector
       storage.get(alias).add(nr);
@@ -178,30 +185,29 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
   }
 
   private void createForwardJoinObject(IntermediateObject intObj, boolean[] nullsArr) throws HiveException {
-    CompositeHiveObject nr = new CompositeHiveObject(totalSz);
+    ArrayList<Object> nr = new ArrayList<Object>(totalSz);
     for (int i = 0; i < numValues; i++) {
       Byte alias = order[i];
       int sz = joinExprs.get(alias).getValueFields().length;
-      if (nullsArr[i])
-        for (int j = 0; j < sz; j++)
-          nr.addHiveObject(null);
-      else
-      {
-        List <SerDeField> fields = joinExprs.get(alias).getListFields();
-        CompositeHiveObject obj = intObj.getObjs()[i];
-        for (int j = 0; j < sz; j++)
-          nr.addHiveObject(obj.get(fields.get(j)));
+      if (nullsArr[i]) {
+        for (int j = 0; j < sz; j++) {
+          nr.add(null);
+        }
+      } else {
+        ArrayList<Object> obj = intObj.getObjs()[i];
+        for (int j = 0; j < sz; j++) {
+          nr.add(obj.get(j));
+        }
       }
     }
-
-    forward(nr);
+    forward(nr, joinOutputObjectInspector);
   }
 
   private void copyOldArray(boolean[] src, boolean[] dest) {
     for (int i = 0; i < src.length; i++) dest[i] = src[i];
   }
 
-  private Vector<boolean[]> joinObjectsInnerJoin(Vector<boolean[]> resNulls, Vector<boolean[]> inputNulls, CompositeHiveObject newObj, IntermediateObject intObj, int left, boolean newObjNull)
+  private Vector<boolean[]> joinObjectsInnerJoin(Vector<boolean[]> resNulls, Vector<boolean[]> inputNulls, ArrayList<Object> newObj, IntermediateObject intObj, int left, boolean newObjNull)
   {
     if (newObjNull) return resNulls;
     Iterator<boolean[]> nullsIter = inputNulls.iterator();
@@ -219,7 +225,7 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
     return resNulls;
   }
   
-  private Vector<boolean[]> joinObjectsLeftOuterJoin(Vector<boolean[]> resNulls, Vector<boolean[]> inputNulls, CompositeHiveObject newObj, IntermediateObject intObj, int left, boolean newObjNull)
+  private Vector<boolean[]> joinObjectsLeftOuterJoin(Vector<boolean[]> resNulls, Vector<boolean[]> inputNulls, ArrayList<Object> newObj, IntermediateObject intObj, int left, boolean newObjNull)
   {
     Iterator<boolean[]> nullsIter = inputNulls.iterator();
     while (nullsIter.hasNext())
@@ -237,7 +243,7 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
     return resNulls;
   }
 
-  private Vector<boolean[]> joinObjectsRightOuterJoin(Vector<boolean[]> resNulls, Vector<boolean[]> inputNulls, CompositeHiveObject newObj, IntermediateObject intObj, int left, boolean newObjNull)
+  private Vector<boolean[]> joinObjectsRightOuterJoin(Vector<boolean[]> resNulls, Vector<boolean[]> inputNulls, ArrayList<Object> newObj, IntermediateObject intObj, int left, boolean newObjNull)
   {
     if (newObjNull) return resNulls;
     boolean allOldObjsNull = true;
@@ -276,7 +282,7 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
     return resNulls;
   }
 
-  private Vector<boolean[]> joinObjectsFullOuterJoin(Vector<boolean[]> resNulls, Vector<boolean[]> inputNulls, CompositeHiveObject newObj, IntermediateObject intObj, int left, boolean newObjNull)
+  private Vector<boolean[]> joinObjectsFullOuterJoin(Vector<boolean[]> resNulls, Vector<boolean[]> inputNulls, ArrayList<Object> newObj, IntermediateObject intObj, int left, boolean newObjNull)
   {
     if (newObjNull) {
       Iterator<boolean[]> nullsIter = inputNulls.iterator();
@@ -344,7 +350,7 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
    * list of nulls is changed appropriately. The list will contain all non-nulls
    * for a inner join. The outer joins are processed appropriately.
    */
-  private Vector<boolean[]> joinObjects(Vector<boolean[]> inputNulls, CompositeHiveObject newObj, IntermediateObject intObj, int joinPos)
+  private Vector<boolean[]> joinObjects(Vector<boolean[]> inputNulls, ArrayList<Object> newObj, IntermediateObject intObj, int joinPos)
   {
     Vector<boolean[]> resNulls = new Vector<boolean[]>();
     boolean newObjNull = newObj == dummyObj[joinPos] ? true : false;
@@ -395,11 +401,11 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
   private void genObject(Vector<boolean[]> inputNulls, int aliasNum, IntermediateObject intObj) 
     throws HiveException {
     if (aliasNum < numValues) {
-      Iterator<CompositeHiveObject> aliasRes = storage.get(order[aliasNum])
+      Iterator<ArrayList<Object>> aliasRes = storage.get(order[aliasNum])
         .iterator();
       iterators.push(aliasRes);
       while (aliasRes.hasNext()) {
-        CompositeHiveObject newObj = aliasRes.next();
+        ArrayList<Object> newObj = aliasRes.next();
         intObj.pushObj(newObj);
         Vector<boolean[]> newNulls = joinObjects(inputNulls, newObj, intObj, aliasNum);
         genObject(newNulls, aliasNum + 1, intObj);
@@ -424,20 +430,24 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
    */
   public void endGroup() throws HiveException {
     try {
-      l4j.trace("Join Op: endGroup called");
+      LOG.trace("Join Op: endGroup called: numValues=" + numValues);
 
       // does any result need to be emitted
       for (int i = 0; i < numValues; i++) {
         Byte alias = order[i];
         if (storage.get(alias).iterator().hasNext() == false) {
-          if (noOuterJoin)
+          if (noOuterJoin) {
+            LOG.trace("No data for alias=" + i);
             return;
-          else
+          } else {
             storage.put(alias, dummyObjVectors[i]);
+          }
         }
       }
 
-      genObject(null, 0, new IntermediateObject(new CompositeHiveObject[numValues], 0));
+      LOG.trace("calling genObject");
+      genObject(null, 0, new IntermediateObject(new ArrayList[numValues], 0));
+      LOG.trace("called genObject");
     } catch (Exception e) {
       e.printStackTrace();
       throw new HiveException(e);
@@ -449,7 +459,7 @@ public class JoinOperator extends Operator<joinDesc> implements Serializable {
    * 
    */
   public void close(boolean abort) throws HiveException {
-    l4j.trace("Join Op close");
+    LOG.trace("Join Op close");
     super.close(abort);
   }
 }

+ 0 - 60
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/LabeledCompositeHiveObject.java

@@ -1,60 +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.ql.exec;
-
-import org.apache.hadoop.hive.serde.*;
-/**
- * wrapper over composite hive object that attaches names to each field
- * (instead of the positional names of CompositeHiveObject)
- */
-public class LabeledCompositeHiveObject extends CompositeHiveObject {
-  String [] labels;
-
-  public LabeledCompositeHiveObject(int width) {
-    super(width);
-    throw new RuntimeException ("Labaled Hive Objects require field names");
-  }
-
-  public LabeledCompositeHiveObject(String [] labels) {
-    super(labels.length);
-    this.labels = labels;
-  }
-
-  @Override
-  public SerDeField getFieldFromExpression(String expr) {
-
-    int dot = expr.indexOf(".");
-    String label = expr;
-    if(dot != -1) {
-      assert(dot != (expr.length()-1));
-
-      label = expr.substring(0, dot);
-      expr =  expr.substring(dot+1);
-    } else {
-      expr = null;
-    }
-
-    for(int i=0; i<width; i++) {
-      if(label.equals(labels[i])) {
-        return new CompositeSerDeField(i, expr);
-      }
-    }
-    throw new RuntimeException ("Cannot match expression "+label+"."+expr+" against any label!");
-  }
-}

+ 56 - 30
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java

@@ -31,10 +31,12 @@ import org.apache.hadoop.hive.ql.plan.mapredWork;
 import org.apache.hadoop.hive.ql.plan.tableDesc;
 import org.apache.hadoop.hive.ql.plan.partitionDesc;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.serde.ConstantTypedSerDeField;
-import org.apache.hadoop.hive.serde.SerDe;
-import org.apache.hadoop.hive.serde.SerDeException;
-import org.apache.hadoop.hive.serde.SerDeField;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 
 
 /**
@@ -48,9 +50,16 @@ public class MapOperator extends Operator <mapredWork> implements Serializable {
   private static final long serialVersionUID = 1L;
   public static enum Counter {DESERIALIZE_ERRORS}
   transient private LongWritable deserialize_error_count = new LongWritable ();
-  transient private SerDe decoder;
-  transient private ArrayList<String> partCols;
-  transient private ArrayList<SerDeField> partFields;
+  transient private Deserializer deserializer;
+  
+  transient private Object row;
+  transient private Object[] rowWithPart;
+  transient private StructObjectInspector rowObjectInspector;
+
+  transient private List<String> partNames;
+  transient private List<String> partValues;
+  transient private List<ObjectInspector> partObjectInspectors;
+  
 
   public void initialize(Configuration hconf) throws HiveException {
     super.initialize(hconf);
@@ -66,12 +75,12 @@ public class MapOperator extends Operator <mapredWork> implements Serializable {
         // pick up work corresponding to this configuration path
         List<String> aliases = conf.getPathToAliases().get(onefile);
         for(String onealias: aliases) {
-          l4j.info("Adding alias " + onealias + " to work list for file " + fpath.toUri().getPath());
+          LOG.info("Adding alias " + onealias + " to work list for file " + fpath.toUri().getPath());
           todo.add(conf.getAliasToWork().get(onealias));
         }
 
         // initialize decoder once based on what table we are processing
-        if(decoder != null) {
+        if(deserializer != null) {
           continue;
         }
 
@@ -83,7 +92,7 @@ public class MapOperator extends Operator <mapredWork> implements Serializable {
         HiveConf.setVar(hconf, HiveConf.ConfVars.HIVETABLENAME, String.valueOf(p.getProperty("name")));
         HiveConf.setVar(hconf, HiveConf.ConfVars.HIVEPARTITIONNAME, String.valueOf(partSpec));
         try {
-          Class sdclass = td.getSerdeClass();
+          Class sdclass = td.getDeserializerClass();
           if(sdclass == null) {
             String className = td.getSerdeClassName();
             if ((className == "") || (className == null)) {
@@ -91,28 +100,40 @@ public class MapOperator extends Operator <mapredWork> implements Serializable {
             }
             sdclass = MapOperator.class.getClassLoader().loadClass(className);
           }
-          decoder = (SerDe) sdclass.newInstance();
-          decoder.initialize(hconf, p);
-
+          deserializer = (Deserializer) sdclass.newInstance();
+          deserializer.initialize(hconf, p);
+          rowObjectInspector = (StructObjectInspector)deserializer.getObjectInspector();
+          
           // Next check if this table has partitions and if so
           // get the list of partition names as well as allocate
           // the serdes for the partition columns
           String pcols = p.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS);
           if (pcols != null && pcols.length() > 0) {
-            partCols = new ArrayList<String>();
-            partFields = new ArrayList<SerDeField>();
-            String[] part_keys = pcols.trim().split("/");
-            for(String key: part_keys) {
-              partCols.add(key);
-              partFields.add(new ConstantTypedSerDeField(key, partSpec.get(key)));
+            partNames = new ArrayList<String>();
+            partValues = new ArrayList<String>();
+            partObjectInspectors = new ArrayList<ObjectInspector>();
+            String[] partKeys = pcols.trim().split("/");
+            for(String key: partKeys) {
+              partNames.add(key);
+              partValues.add(partSpec.get(key));
+              partObjectInspectors.add(
+                  ObjectInspectorFactory.getStandardPrimitiveObjectInspector(String.class));
             }
+            StructObjectInspector partObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(partNames, partObjectInspectors);
+            
+            rowWithPart = new Object[2];
+            rowWithPart[1] = partValues;
+            rowObjectInspector = ObjectInspectorFactory.getUnionStructObjectInspector(
+                Arrays.asList(new StructObjectInspector[]{
+                    rowObjectInspector, 
+                    partObjectInspector}));
           }
           else {
-            partCols = null;
-            partFields = null;
+            partNames = null;
+            partValues = null;
           }
 
-          l4j.info("Got partitions: " + pcols);
+          LOG.info("Got partitions: " + pcols);
         } catch (SerDeException e) {
           e.printStackTrace();
           throw new HiveException (e);
@@ -129,7 +150,7 @@ public class MapOperator extends Operator <mapredWork> implements Serializable {
     if(todo.size() == 0) {
       // didn't find match for input file path in configuration!
       // serious problem ..
-      l4j.error("Configuration does not have any alias for path: " + fpath.toUri().getPath());
+      LOG.error("Configuration does not have any alias for path: " + fpath.toUri().getPath());
       throw new HiveException("Configuration and input path are inconsistent");
     }
 
@@ -146,19 +167,24 @@ public class MapOperator extends Operator <mapredWork> implements Serializable {
     }
   }
 
-  public void process(HiveObject r) throws HiveException {
-    throw new RuntimeException("Should not be called!");
-  }
-
   public void process(Writable value) throws HiveException {
     try {
-      Object ev = decoder.deserialize(value);
-      HiveObject ho = new TableHiveObject(ev, decoder, partCols, partFields);
-      forward(ho);
+      if (partNames == null) {
+        row = deserializer.deserialize(value);
+        forward(row, rowObjectInspector);
+      } else {
+        rowWithPart[0] = deserializer.deserialize(value);
+        forward(rowWithPart, rowObjectInspector);
+      }
     } catch (SerDeException e) {
       // TODO: policy on deserialization errors
       deserialize_error_count.set(deserialize_error_count.get()+1);
       throw new HiveException (e);
     }
   }
+
+  public void process(Object row, ObjectInspector rowInspector)
+      throws HiveException {
+    throw new HiveException("Hive 2 Internal error: should not be called!");
+  }
 }

+ 1 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java

@@ -42,6 +42,7 @@ public class MapRedTask extends Task<mapredWork> implements Serializable {
   public int execute() {
 
     try {
+      // enable assertion
       String hadoopExec = conf.getVar(HiveConf.ConfVars.HADOOPBIN);
       String hiveJar = conf.getJar();
       String hiveConfArgs = ExecDriver.generateCmdLine(conf);

+ 0 - 57
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/NullHiveObject.java

@@ -1,57 +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.ql.exec;
-
-import java.util.*;
-
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * Represents a NULL object
- */
-public class NullHiveObject extends HiveObject {
-
-  public NullHiveObject() {
-    setIsNull(true);
-  }
-
-  public SerDeField getFieldFromExpression(String expr) throws HiveException {
-    return null;
-  }
-
-  public HiveObject get(SerDeField field) throws HiveException {
-    return this;
-  }
-
-  public List<SerDeField> getFields() throws HiveException {
-    return null;
-  }
-
-  public boolean isPrimitive() { return false;}
-
-  public boolean equals(Object other) {
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    throw new RuntimeException("not supported");
-  }
-}

+ 17 - 17
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java

@@ -23,6 +23,8 @@ import java.io.*;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.mapredWork;
+import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.conf.Configuration;
@@ -86,7 +88,7 @@ public abstract class Operator <T extends Serializable> implements Serializable
 
   transient protected HashMap<Enum<?>, LongWritable> statsMap = new HashMap<Enum<?>, LongWritable> ();
   transient protected OutputCollector out;
-  transient protected Log l4j;
+  transient protected Log LOG = LogFactory.getLog(this.getClass().getName());;
   transient protected mapredWork gWork;
   transient protected String alias;
   transient protected String joinAlias;
@@ -159,50 +161,47 @@ public abstract class Operator <T extends Serializable> implements Serializable
   }
 
   public void initialize (Configuration hconf) throws HiveException {
-    l4j = LogFactory.getLog(this.getClass().getName());
-    l4j.info("Initializing Self");
+    LOG.info("Initializing Self");
     
     if(childOperators == null) {
       return;
     }
-    l4j.info("Initializing children:");
+    LOG.info("Initializing children:");
     for(Operator<? extends Serializable> op: childOperators) {
       op.initialize(hconf);
     }    
-    l4j.info("Initialization Done");
+    LOG.info("Initialization Done");
   }
 
-  public abstract void process(HiveObject r) throws HiveException;
+  public abstract void process(Object row, ObjectInspector rowInspector) throws HiveException;
  
   // If a operator wants to do some work at the beginning of a group
   public void startGroup() throws HiveException {
-    l4j = LogFactory.getLog(this.getClass().getName());
-    l4j.trace("Starting group");
+    LOG.debug("Starting group");
     
     if (childOperators == null)
       return;
     
-    l4j.trace("Starting group for children:");
+    LOG.debug("Starting group for children:");
     for (Operator<? extends Serializable> op: childOperators)
       op.startGroup();
     
-    l4j.trace("Start group Done");
+    LOG.debug("Start group Done");
   }  
   
   // If a operator wants to do some work at the beginning of a group
   public void endGroup() throws HiveException
   {
-     l4j = LogFactory.getLog(this.getClass().getName());
-    l4j.trace("Ending group");
+    LOG.debug("Ending group");
     
     if (childOperators == null)
       return;
     
-    l4j.trace("Ending group for children:");
+    LOG.debug("Ending group for children:");
     for (Operator<? extends Serializable> op: childOperators)
       op.endGroup();
     
-    l4j.trace("Start group Done");
+    LOG.debug("End group Done");
   }
 
   public void close(boolean abort) throws HiveException {
@@ -218,12 +217,13 @@ public abstract class Operator <T extends Serializable> implements Serializable
     }
   }
 
-  protected void forward(HiveObject r) throws HiveException {
+  protected void forward(Object row, ObjectInspector rowInspector) throws HiveException {
+    
     if(childOperators == null) {
       return;
     }
     for(Operator<? extends Serializable> o: childOperators) {
-      o.process(r);
+      o.process(row, rowInspector);
     }
   }
 
@@ -249,7 +249,7 @@ public abstract class Operator <T extends Serializable> implements Serializable
 
   public void logStats () {
     for(Enum<?> e: statsMap.keySet()) {
-      l4j.info(e.toString() + ":" + statsMap.get(e).toString());
+      LOG.info(e.toString() + ":" + statsMap.get(e).toString());
     }    
   }
 

+ 0 - 67
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/PrimitiveHiveObject.java

@@ -1,67 +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.ql.exec;
-
-import java.util.*;
-
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * Encapsulation for a primitive Java Object
- */
-
-public final class PrimitiveHiveObject extends HiveObject {
-
-  public PrimitiveHiveObject(Object javaObject) {
-
-    this.javaObject = javaObject;
-  }
-
-  public SerDeField getFieldFromExpression(String expr) throws HiveException {
-    throw new HiveException ("Illegal call getFieldFromExpression() on Primitive Object");
-  }
-
-  public HiveObject get(SerDeField field) throws HiveException {
-    throw new HiveException ("Illegal call get() on Primitive Object");
-  }
-
-  public List<SerDeField> getFields() throws HiveException {
-    throw new HiveException ("Illegal call getFields() on Primitive Object");
-  }
-
-  public boolean isPrimitive() { return true; }
-
-  @Override
-  public String toString () {
-    return (javaObject == null ? "" : javaObject.toString());
-  }
-
-  @Override
-  public int hashCode() {
-    return (javaObject == null ? 0 : javaObject.hashCode());
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (! (other instanceof PrimitiveHiveObject)) return false;
-    return javaObject == null ? ((PrimitiveHiveObject)other).javaObject == null
-        : javaObject.equals(((PrimitiveHiveObject)other).javaObject);
-  }
-}

+ 90 - 38
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java

@@ -19,12 +19,22 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import java.io.*;
+import java.util.ArrayList;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.PlanUtils;
 import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.reduceSinkDesc;
+import org.apache.hadoop.hive.ql.plan.tableDesc;
 import org.apache.hadoop.hive.ql.io.*;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.Serializer;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.io.Text;
 
 /**
  * Reduce Sink Operator sends output to the reduce stage
@@ -34,10 +44,14 @@ public class ReduceSinkOperator extends TerminalOperator <reduceSinkDesc> implem
   private static final long serialVersionUID = 1L;
   transient protected ExprNodeEvaluator[] keyEval;
   transient protected ExprNodeEvaluator[] valueEval;
-  transient WritableComparableHiveObject wcho;
-  transient WritableHiveObject who;
-  transient boolean keyIsSingleton, valueIsSingleton;
-
+  
+  // TODO: we use MetadataTypedColumnsetSerDe for now, till DynamicSerDe is ready
+  transient Serializer keySerializer;
+  transient Serializer valueSerializer;
+  transient int tag;
+  transient byte[] tagByte = new byte[1];
+  transient int numPartitionFields; 
+  
   public void initialize(Configuration hconf) throws HiveException {
     super.initialize(hconf);
     try {
@@ -46,61 +60,99 @@ public class ReduceSinkOperator extends TerminalOperator <reduceSinkDesc> implem
       for(exprNodeDesc e: conf.getKeyCols()) {
         keyEval[i++] = ExprNodeEvaluatorFactory.get(e);
       }
-      keyIsSingleton = false; //(i == 1);
 
       valueEval = new ExprNodeEvaluator[conf.getValueCols().size()];
       i=0;
       for(exprNodeDesc e: conf.getValueCols()) {
         valueEval[i++] = ExprNodeEvaluatorFactory.get(e);
       }
-      valueIsSingleton = false; //(i == 1);
 
-      // TODO: Use NaiiveSerializer for now 
-      // Once we make sure CompositeHiveObject.getJavaObject() returns a Java List,
-      // we will use MetadataTypedSerDe to serialize the data, instead of using
-      // NaiiveSerializer.
-      int tag = conf.getTag();
-      if(tag == -1) {
-        who = new NoTagWritableHiveObject(null, new NaiiveSerializer());
-        wcho = new NoTagWritableComparableHiveObject(null, new NaiiveSerializer());
-        l4j.info("Using tag = -1");
-      } else {
-        l4j.info("Using tag = " + tag);
-        who = new WritableHiveObject(tag, null, new NaiiveSerializer());
-        wcho = new WritableComparableHiveObject(tag, null, new NaiiveSerializer());
-      }
+      tag = conf.getTag();
+      tagByte[0] = (byte)tag;
+      LOG.info("Using tag = " + tag);
 
+      tableDesc keyTableDesc = conf.getKeySerializeInfo();
+      keySerializer = (Serializer)keyTableDesc.getDeserializerClass().newInstance();
+      keySerializer.initialize(null, keyTableDesc.getProperties());
+      
+      tableDesc valueTableDesc = conf.getValueSerializeInfo();
+      valueSerializer = (Serializer)valueTableDesc.getDeserializerClass().newInstance();
+      valueSerializer.initialize(null, valueTableDesc.getProperties());
+      
       // Set the number of key fields to be used in the partitioner.
-      WritableComparableHiveObject.setNumPartitionFields(conf.getNumPartitionFields());
+      numPartitionFields = conf.getNumPartitionFields();
     } catch (Exception e) {
       e.printStackTrace();
       throw new RuntimeException(e);
     }
   }
 
-  public void process(HiveObject r) throws HiveException {
-    if(keyIsSingleton) {
-      wcho.setHo(keyEval[0].evaluate(r));
-    } else {
-      CompositeHiveObject nr = new CompositeHiveObject (keyEval.length);
+  transient InspectableObject tempInspectableObject = new InspectableObject();
+  transient HiveKey keyWritable = new HiveKey();
+  transient Text valueText;
+  
+  transient ObjectInspector keyObjectInspector;
+  transient ObjectInspector valueObjectInspector;
+  transient ArrayList<ObjectInspector> keyFieldsObjectInspectors = new ArrayList<ObjectInspector>();
+  transient ArrayList<ObjectInspector> valueFieldsObjectInspectors = new ArrayList<ObjectInspector>();
+  
+  public void process(Object row, ObjectInspector rowInspector) throws HiveException {
+    // TODO: use DynamicSerDe when that is ready
+    try {
+      // Generate hashCode for the tuple
+      int keyHashCode = 0;
+      if (numPartitionFields == -1) {
+        keyHashCode = (int)(Math.random() * Integer.MAX_VALUE);
+      }
+      ArrayList<Object> keys = new ArrayList<Object>(keyEval.length);
       for(ExprNodeEvaluator e: keyEval) {
-        nr.addHiveObject(e.evaluate(r));
+        e.evaluate(row, rowInspector, tempInspectableObject);
+        keys.add(tempInspectableObject.o);
+        if (numPartitionFields == keys.size()) {
+          keyHashCode = keys.hashCode();
+        }
+        if (keyObjectInspector == null) {
+          keyFieldsObjectInspectors.add(tempInspectableObject.oi);
+        }
       }
-      wcho.setHo(nr);
-    }
-
-    if(valueIsSingleton) {
-      who.setHo(valueEval[0].evaluate(r));
-    } else {
-      CompositeHiveObject nr = new CompositeHiveObject (valueEval.length);
+      if (numPartitionFields > keys.size()) {
+        keyHashCode = keys.hashCode();
+      }
+      if (keyObjectInspector == null) {
+        keyObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(
+            ObjectInspectorUtils.getIntegerArray(keyFieldsObjectInspectors.size()),
+            keyFieldsObjectInspectors);
+      }
+      Text key = (Text)keySerializer.serialize(keys, keyObjectInspector);
+      if (tag == -1) {
+        keyWritable.set(key.getBytes(), 0, key.getLength());
+      } else {
+        int keyLength = key.getLength();
+        keyWritable.setSize(keyLength+1);
+        System.arraycopy(key.getBytes(), 0, keyWritable.get(), 0, keyLength);
+        keyWritable.get()[keyLength] = tagByte[0];
+      }
+      keyWritable.setHashCode(keyHashCode);
+      
+      ArrayList<String> values = new ArrayList<String>(valueEval.length);
       for(ExprNodeEvaluator e: valueEval) {
-        nr.addHiveObject(e.evaluate(r));
+        e.evaluate(row, rowInspector, tempInspectableObject);
+        values.add(tempInspectableObject.o == null ? null : tempInspectableObject.o.toString());
+        if (valueObjectInspector == null) {
+          valueFieldsObjectInspectors.add(tempInspectableObject.oi);
+        }
+      }
+      if (valueObjectInspector == null) {
+        valueObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(
+            ObjectInspectorUtils.getIntegerArray(valueFieldsObjectInspectors.size()),
+            valueFieldsObjectInspectors);
       }
-      who.setHo(nr);
+      valueText = (Text)valueSerializer.serialize(values, valueObjectInspector);
+    } catch (SerDeException e) {
+      throw new HiveException(e);
     }
-
     try {
-      out.collect(wcho, who);
+      out.collect(keyWritable, valueText);
     } catch (IOException e) {
       throw new HiveException (e);
     }

+ 42 - 38
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java

@@ -25,10 +25,12 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.plan.scriptDesc;
-import org.apache.hadoop.hive.ql.plan.tableDesc;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.io.*;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.Serializer;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.mapred.LineRecordReader.LineReader;
 import org.apache.hadoop.util.StringUtils;
@@ -50,8 +52,10 @@ public class ScriptOperator extends Operator<scriptDesc> implements Serializable
   transient Thread errThread;
   transient Process scriptPid;
   transient Configuration hconf;
-  transient SerDe decoder;
-  transient HiveObjectSerializer hos;
+  // Input to the script
+  transient Serializer scriptInputSerializer;
+  // Output from the script
+  transient Deserializer scriptOutputDeserializer;
   transient volatile Throwable scriptError = null;
 
   /**
@@ -93,24 +97,18 @@ public class ScriptOperator extends Operator<scriptDesc> implements Serializable
     try {
       this.hconf = hconf;
 
-      tableDesc td = conf.getScriptOutputInfo();
-      if(td == null) {
-        td = Utilities.defaultTabTd;
-      }
-      decoder = td.getSerdeClass().newInstance();
-      decoder.initialize(hconf, td.getProperties());
+      scriptOutputDeserializer = conf.getScriptOutputInfo().getDeserializerClass().newInstance();
+      scriptOutputDeserializer.initialize(hconf, conf.getScriptOutputInfo().getProperties());
 
-      hos = new NaiiveJSONSerializer();
-      Properties p = new Properties ();
-      p.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, ""+Utilities.tabCode);
-      hos.initialize(p);
+      scriptInputSerializer = (Serializer)conf.getScriptInputInfo().getDeserializerClass().newInstance();
+      scriptInputSerializer.initialize(hconf, conf.getScriptInputInfo().getProperties());
 
       String [] cmdArgs = splitArgs(conf.getScriptCmd());
       String [] wrappedCmdArgs = addWrapper(cmdArgs);
-      l4j.info("Executing " + Arrays.asList(wrappedCmdArgs));
-      l4j.info("tablename=" + hconf.get(HiveConf.ConfVars.HIVETABLENAME.varname));
-      l4j.info("partname=" + hconf.get(HiveConf.ConfVars.HIVEPARTITIONNAME.varname));
-      l4j.info("alias=" + alias);
+      LOG.info("Executing " + Arrays.asList(wrappedCmdArgs));
+      LOG.info("tablename=" + hconf.get(HiveConf.ConfVars.HIVETABLENAME.varname));
+      LOG.info("partname=" + hconf.get(HiveConf.ConfVars.HIVEPARTITIONNAME.varname));
+      LOG.info("alias=" + alias);
 
       ProcessBuilder pb = new ProcessBuilder(wrappedCmdArgs);
       Map<String, String> env = pb.environment();
@@ -121,7 +119,8 @@ public class ScriptOperator extends Operator<scriptDesc> implements Serializable
       scriptOut = new DataOutputStream(new BufferedOutputStream(scriptPid.getOutputStream()));
       scriptIn = new DataInputStream(new BufferedInputStream(scriptPid.getInputStream()));
       scriptErr = new DataInputStream(new BufferedInputStream(scriptPid.getErrorStream()));
-      outThread = new StreamThread(scriptIn, new OutputStreamProcessor (), "OutputProcessor");
+      outThread = new StreamThread(scriptIn, new OutputStreamProcessor(
+          scriptOutputDeserializer.getObjectInspector()), "OutputProcessor");
       outThread.start();
       errThread = new StreamThread(scriptErr, new ErrorStreamProcessor (), "ErrorProcessor");
       errThread.start();
@@ -132,14 +131,22 @@ public class ScriptOperator extends Operator<scriptDesc> implements Serializable
     }
   }
 
-  public void process(HiveObject r) throws HiveException {
+  Text text = new Text();
+  public void process(Object row, ObjectInspector rowInspector) throws HiveException {
     if(scriptError != null) {
       throw new HiveException(scriptError);
     }
     try {
-      hos.serialize(r, scriptOut);
+      text = (Text) scriptInputSerializer.serialize(row, rowInspector);
+      scriptOut.write(text.getBytes(), 0, text.getLength());
+      scriptOut.write(Utilities.newLineCode);
+    } catch (SerDeException e) {
+      LOG.error("Error in serializing the row: " + e.getMessage());
+      scriptError = e;
+      serialize_error_count.set(serialize_error_count.get() + 1);
+      throw new HiveException(e);
     } catch (IOException e) {
-      l4j.error("Error in writing to script: " + e.getMessage());
+      LOG.error("Error in writing to script: " + e.getMessage());
       scriptError = e;
       throw new HiveException(e);
     }
@@ -155,7 +162,7 @@ public class ScriptOperator extends Operator<scriptDesc> implements Serializable
         scriptOut.close();
         int exitVal = scriptPid.waitFor();
         if (exitVal != 0) {
-          l4j.error("Script failed with code " + exitVal);
+          LOG.error("Script failed with code " + exitVal);
           new_abort = true;
         };
       } catch (IOException e) {
@@ -163,11 +170,6 @@ public class ScriptOperator extends Operator<scriptDesc> implements Serializable
       } catch (InterruptedException e) { }
     }
 
-    // the underlying hive object serializer keeps track of serialization errors
-    if(hos != null) {
-      serialize_error_count.set(hos.getWriteErrorCount());
-    }
-
     try {
       // try these best effort
       outThread.join(0);
@@ -190,17 +192,19 @@ public class ScriptOperator extends Operator<scriptDesc> implements Serializable
 
 
   class OutputStreamProcessor implements StreamProcessor {
-    public OutputStreamProcessor () {}
+    Object row;
+    ObjectInspector rowInspector;
+    public OutputStreamProcessor(ObjectInspector rowInspector) {
+      this.rowInspector = rowInspector;
+    }
     public void processLine(Text line) throws HiveException {
-      HiveObject ho;
       try {
-        Object ev = decoder.deserialize(line);
-        ho = new TableHiveObject(ev, decoder);
+        row = scriptOutputDeserializer.deserialize(line);
       } catch (SerDeException e) {
         deserialize_error_count.set(deserialize_error_count.get()+1);
         return;
       }
-      forward(ho);
+      forward(row, rowInspector);
     }
     public void close() {
     }
@@ -243,11 +247,11 @@ public class ScriptOperator extends Operator<scriptDesc> implements Serializable
           }
           proc.processLine(row);
         }
-        l4j.info("StreamThread "+name+" done");
+        LOG.info("StreamThread "+name+" done");
 
       } catch (Throwable th) {
         scriptError = th;
-        l4j.warn(StringUtils.stringifyException(th));
+        LOG.warn(StringUtils.stringifyException(th));
       } finally {
         try {
           if(lineReader != null) {
@@ -256,8 +260,8 @@ public class ScriptOperator extends Operator<scriptDesc> implements Serializable
           in.close();
           proc.close();
         } catch (Exception e) {
-          l4j.warn(name + ": error in closing ..");
-          l4j.warn(StringUtils.stringifyException(e));
+          LOG.warn(name + ": error in closing ..");
+          LOG.warn(StringUtils.stringifyException(e));
         }
       }
     }

+ 42 - 10
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java

@@ -19,10 +19,14 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import java.io.*;
+import java.util.ArrayList;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.selectDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -33,26 +37,54 @@ public class SelectOperator extends Operator <selectDesc> implements Serializabl
   private static final long serialVersionUID = 1L;
   transient protected ExprNodeEvaluator[] eval;
 
+  transient ArrayList<Object> output;
+  transient ArrayList<ObjectInspector> outputFieldObjectInspectors;
+  transient ObjectInspector outputObjectInspector;
+  transient InspectableObject tempInspectableObject;
+  
+  boolean firstRow;
+  
   public void initialize(Configuration hconf) throws HiveException {
     super.initialize(hconf);
     try {
-      eval = new ExprNodeEvaluator[conf.getColList().size()];
-      int i=0;
-      for(exprNodeDesc e: conf.getColList()) {
-        eval[i++] = ExprNodeEvaluatorFactory.get(e);
+      ArrayList<exprNodeDesc> colList = conf.getColList();
+      eval = new ExprNodeEvaluator[colList.size()];
+      for(int i=0; i<colList.size(); i++) {
+        assert(colList.get(i) != null);
+        eval[i] = ExprNodeEvaluatorFactory.get(colList.get(i));
       }
+      output = new ArrayList<Object>(eval.length);
+      outputFieldObjectInspectors = new ArrayList<ObjectInspector>(eval.length);
+      for(int j=0; j<eval.length; j++) {
+        output.add(null);
+        outputFieldObjectInspectors.add(null);
+      }
+      tempInspectableObject = new InspectableObject();      
+      firstRow = true;
     } catch (Exception e) {
       e.printStackTrace();
       throw new RuntimeException(e);
     }
   }
 
-  public void process(HiveObject r) throws HiveException {
-    CompositeHiveObject nr = new CompositeHiveObject (eval.length);
-    for(ExprNodeEvaluator e: eval) {
-      HiveObject ho = e.evaluate(r);
-      nr.addHiveObject(ho);
+  public void process(Object row, ObjectInspector rowInspector)
+      throws HiveException {
+    for(int i=0; i<eval.length; i++) {
+      eval[i].evaluate(row, rowInspector, tempInspectableObject);
+      output.set(i, tempInspectableObject.o);
+      if (firstRow) {
+        outputFieldObjectInspectors.set(i, tempInspectableObject.oi);
+      }
+    }
+    if (firstRow) {
+      firstRow = false;
+      ArrayList<String> fieldNames = new ArrayList<String>(eval.length);
+      for(int i=0; i<eval.length; i++) {
+        fieldNames.add(Integer.valueOf(i).toString());
+      }
+      outputObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(
+        fieldNames, outputFieldObjectInspectors);
     }
-    forward(nr);
+    forward(output, outputObjectInspector);
   }
 }

+ 0 - 167
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/TableHiveObject.java

@@ -1,167 +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.ql.exec;
-
-import java.util.*;
-
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * A TableHiveObject is Hive encapsulation of Objects returned from a table
- * It allows navigation using the SerDe associated with the Table. They also
- * encapsulate the position relative to the row object that the current object
- * represents.
- */
-public class TableHiveObject extends HiveObject {
-
-  // References to the table and initial table Object
-  protected SerDe tableSerDe;
-  protected Object rowObject;
-  protected ArrayList<String> partCols;
-  protected ArrayList<SerDeField> partFields;
-
-  // References to the field of the row that the HiveObject refers to
-  protected SerDeField myField;
-
-  public TableHiveObject(Object javaObject, SerDe tableSerDe) {
-      if(javaObject == null) {
-          throw new RuntimeException("javaObject may not be null in TableHiveObject constructor");
-      }
-
-
-    this.javaObject = javaObject;
-    this.tableSerDe = tableSerDe;
-    this.rowObject = javaObject;
-    this.partCols = null;
-    this.partFields = null;
-  }
-
-
-  public TableHiveObject(Object javaObject, SerDe tableSerDe,
-                         ArrayList<String> partCols,
-                         ArrayList<SerDeField> partFields) {
-      if(javaObject == null) {
-          throw new RuntimeException("javaObject may not be null in TableHiveObject constructor");
-      }
-
-
-    this.javaObject = javaObject;
-    this.tableSerDe = tableSerDe;
-    this.rowObject = javaObject;
-    this.partCols = partCols;
-    this.partFields = partFields;
-  }
-
-  protected TableHiveObject(Object javaObject, SerDe tableSerDe,
-                            SerDeField myField, Object rowObject,
-                            ArrayList<String> partCols,
-                            ArrayList<SerDeField> partFields) {
-    if(javaObject == null) {
-      throw new RuntimeException("javaObject may not be null in TableHiveObject constructor");
-    }
-    this.javaObject = javaObject;
-    this.tableSerDe = tableSerDe;
-    this.myField = myField;
-    this.rowObject = rowObject;
-    this.partCols = partCols;
-    this.partFields = partFields;
-  }
-
-  public SerDeField getFieldFromExpression(String expr) throws HiveException {
-    try {
-      if(expr == null || expr.equals(""))
-        throw new RuntimeException("Need non empty expression");
-
-      // Check if this is a partition column
-      if (partCols != null) {
-        int pos = partCols.indexOf(expr);
-        if (pos != -1) {
-          return partFields.get(pos);
-        }
-      }
-
-      String realExpr;
-      if(myField != null) {
-        if (expr.charAt(0) == '[') {
-          realExpr = myField.getName() + expr;
-        } else {
-          realExpr = myField.getName() + "." + expr;
-        }
-      } else {
-        realExpr = expr;
-      }
-
-      if(!ExpressionUtils.isComplexExpression(realExpr)) {
-        return tableSerDe.getFieldFromExpression(null, realExpr);
-      } else {
-        return new ComplexSerDeField(null, realExpr, tableSerDe);
-      }
-    } catch (SerDeException e) {
-      e.printStackTrace();
-      throw new HiveException (e);
-    }
-  }
-
-  public HiveObject get(SerDeField field) throws HiveException {
-    try {
-      Object o = field.get(rowObject);
-      if (o == null) 
-        return new NullHiveObject();
-
-      if(field.isPrimitive())
-        return new PrimitiveHiveObject(o);
-      else 
-        return new TableHiveObject(o, tableSerDe, field, rowObject, partCols, partFields);
-    } catch (SerDeException e) {
-      throw new HiveException(e);
-    }
-  }
-
-  public List<SerDeField> getFields() throws HiveException {
-    try {
-      return tableSerDe.getFields(myField);
-    } catch (SerDeException e) {
-      e.printStackTrace();
-      throw new HiveException (e);
-    }
-  }
-
-  public boolean isPrimitive() { return false;}
-
-  
-  public String toString() {
-    try {
-      return tableSerDe.toJSONString(myField.get(rowObject), myField);
-    } catch (SerDeException e) {
-      throw new RuntimeException(e); 
-    }
-  }
-  
-  @Override
-  public boolean equals(Object other) {
-    throw new RuntimeException("not supported");
-  }
-
-  @Override
-  public int hashCode() {
-    throw new RuntimeException("not supported");
-  }
-
-}

+ 0 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/UDAF.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 //import org.apache.hadoop.hive.serde.ReflectionSerDe;
-import org.apache.hadoop.hive.serde.SerDeException;
 
 /**
  * Base class for all User-defined Aggregation Function (UDAF) classes.

+ 0 - 2
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/UDF.java

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
 /**
  * A dummy User-defined function (UDF) for the use with Hive.
  * 

+ 76 - 21
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java

@@ -32,7 +32,11 @@ import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.ReflectionUtils;
 
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.plan.*;
@@ -40,8 +44,11 @@ import org.apache.hadoop.hive.ql.plan.PlanUtils.ExpressionTypes;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
 
 @SuppressWarnings("nls")
 public class Utilities {
@@ -49,9 +56,10 @@ public class Utilities {
   /**
    * The object in the reducer are composed of these top level fields
    */
-  public static enum ReduceField { KEY, VALUE, ALIAS };
 
+  public static enum ReduceField { KEY, VALUE, ALIAS };
   private static volatile mapredWork gWork = null;
+  static final private Log LOG = LogFactory.getLog("hive.ql.exec.Utilities");
 
   public static void clearMapRedWork (Configuration job) {
     try {
@@ -113,7 +121,7 @@ public class Utilities {
       protected Expression instantiate(Object oldInstance, Encoder out) {
       return new Expression(Enum.class,
                             "valueOf",
-                            new Object[] { oldInstance.getClass(), ((Enum) oldInstance).name() });
+                            new Object[] { oldInstance.getClass(), ((Enum<?>) oldInstance).name() });
     }
     protected boolean mutatesTo(Object oldInstance, Object newInstance) {
       return oldInstance == newInstance;
@@ -190,21 +198,13 @@ public class Utilities {
   public static tableDesc defaultTd;
   static {
     // by default we expect ^A separated strings
-    Properties p = new Properties ();
-    p.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "" + Utilities.ctrlaCode);
-    defaultTd = new tableDesc(MetadataTypedColumnsetSerDe.class,
-        TextInputFormat.class, 
-        IgnoreKeyTextOutputFormat.class, p);
+    defaultTd = PlanUtils.getDefaultTableDesc("" + Utilities.ctrlaCode);
   }
 
   public static tableDesc defaultTabTd;
   static {
-    // by default we expect ^A separated strings
-    Properties p = new Properties ();
-    p.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
-    defaultTabTd = new tableDesc(MetadataTypedColumnsetSerDe.class,
-        TextInputFormat.class, 
-        IgnoreKeyTextOutputFormat.class, p);
+    // Default tab-separated tableDesc
+    defaultTabTd = PlanUtils.getDefaultTableDesc("" + Utilities.tabCode);
   }
   
   public final static int newLineCode = 10;
@@ -289,7 +289,7 @@ public class Utilities {
   }
 
   public static tableDesc getTableDesc(Table tbl) {
-    return (new tableDesc (tbl.getSerDe().getClass(), tbl.getInputFormatClass(), tbl.getOutputFormatClass(), tbl.getSchema()));
+    return (new tableDesc (tbl.getDeserializer().getClass(), tbl.getInputFormatClass(), tbl.getOutputFormatClass(), tbl.getSchema()));
   }
 
 
@@ -297,12 +297,12 @@ public class Utilities {
     return (new partitionDesc (getTableDesc(part.getTable()), part.getSpec()));
   }
 
-  public static void addMapWork(mapredWork mr, Table tbl, String alias, Operator work) {
+  public static void addMapWork(mapredWork mr, Table tbl, String alias, Operator<?> work) {
     mr.addMapWork(tbl.getDataLocation().getPath(), alias, work, 
                   new partitionDesc(getTableDesc(tbl), null));
   }
 
-  private static String getOpTreeSkel_helper(Operator op, String indent) {
+  private static String getOpTreeSkel_helper(Operator<?> op, String indent) {
     if (op == null)
       return "";
   
@@ -312,13 +312,13 @@ public class Utilities {
     sb.append("\n");
     if (op.getChildOperators() != null)
       for(Object child: op.getChildOperators()) {
-        sb.append(getOpTreeSkel_helper((Operator)child, indent + "  "));
+        sb.append(getOpTreeSkel_helper((Operator<?>)child, indent + "  "));
       }
 
     return sb.toString();
   }
 
-  public static String getOpTreeSkel(Operator op) {
+  public static String getOpTreeSkel(Operator<?> op) {
     return getOpTreeSkel_helper(op, "");
   }
 
@@ -380,4 +380,59 @@ public class Utilities {
 
     return prefix+suffix;
   }
+
+  public final static String NSTR = "";
+  public static enum streamStatus {EOF, TERMINATED, NORMAL}
+  public static streamStatus readColumn(DataInput in, OutputStream out) throws IOException {
+
+    while (true) {
+      int b;
+      try {
+        b = (int)in.readByte();
+      } catch (EOFException e) {
+        return streamStatus.EOF;
+      }
+
+      if (b == Utilities.newLineCode) {
+        return streamStatus.TERMINATED;
+      }
+
+      if (b == Utilities.ctrlaCode) {
+        return streamStatus.NORMAL;
+      }
+
+      out.write(b);
+    }
+    // Unreachable
+  }
+  
+  public static OutputStream createCompressedStream(JobConf jc,
+                                                    OutputStream out) throws IOException {
+    boolean isCompressed = FileOutputFormat.getCompressOutput(jc);
+    if(isCompressed) {
+      Class<? extends CompressionCodec> codecClass =
+        FileOutputFormat.getOutputCompressorClass(jc, DefaultCodec.class);
+      CompressionCodec codec = (CompressionCodec)
+        ReflectionUtils.newInstance(codecClass, jc);
+      return codec.createOutputStream(out);
+    } else {
+      return (out);
+    }
+  }
+
+  public static SequenceFile.Writer createSequenceWriter(JobConf jc, FileSystem fs,
+                                                         Path file, Class<?> keyClass,
+                                                         Class<?> valClass) throws IOException {
+    CompressionCodec codec = null;
+    CompressionType compressionType = CompressionType.NONE;
+    if (SequenceFileOutputFormat.getCompressOutput(jc)) {
+      compressionType = SequenceFileOutputFormat.getOutputCompressionType(jc);
+      Class codecClass = SequenceFileOutputFormat.getOutputCompressorClass(jc, DefaultCodec.class);
+      codec = (CompressionCodec) 
+        ReflectionUtils.newInstance(codecClass, jc);
+    }
+    return (SequenceFile.createWriter(fs, jc, file,
+                                      keyClass, valClass, compressionType, codec));
+
+  }
 }

+ 0 - 96
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/HiveObjectComparator.java

@@ -1,96 +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.ql.io;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.exec.HiveObject;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.plan.mapredWork;
-
-/** 
- * Comparator for WritableComparableHiveObjects
- * 
- * We pass this in explicitly as a output key comparator so that we get a chance to
- * initialize the comparator using the job configuration/Hive plan. This allows us
- * to pick up the right deserializer (if we register with WritableComparator - we get
- * no such shot).
- *
- * This class provides a sort implementation only. The grouping implementation uses
- * the base compare() implementation - which just invokes compareTo() on the underlying
- * WritableComparableHiveObject.
- *
- */ 
-
-public class HiveObjectComparator extends WritableComparator implements Configurable {
-
-  // the serializer used to compare hive objects
-  protected HiveObjectSerializer hos;
-  protected boolean isTagged;
-
-  public Configuration getConf() {
-    throw new RuntimeException ("Unexpected invocation");
-  }
-
-  public void setConf(Configuration conf) {
-    mapredWork gWork = Utilities.getMapRedWork (conf);
-    isTagged = gWork.getNeedsTagging(); 
-    hos = new NaiiveSerializer();
-  }
-
-
-  public HiveObjectComparator(Class c) {
-    super(c);
-  }
-
-
-  public HiveObjectComparator () {
-    this(WritableComparableHiveObject.class);
-  }
-
-  /**
-   * This is the interface used to sort WritableComparableHiveObjects
-   * If the objects are not tagged - then it's simple - we just sort them (for now)
-   * based on the serialized object. If it is tagged - then we need to use the
-   * serialized object as the higher order bits to sort on (so that grouping is
-   * maintained) and then use the tag to break the tie (so that things are ordered
-   * by tag in the same co-group
-   */
-  public int compare(byte[] b1, int s1, int l1,
-                     byte[] b2, int s2, int l2) {
-    if(!isTagged) {
-      return (hos.compare(b1, s1, l1, b2, s2, l2));
-    } else {
-      int ret = hos.compare(b1, s1+1, l1-1, b2, s2+1, l2-1);
-      if(ret == 0) {
-        // use tag to break tie
-        ret = ((int)(b1[s1] & 0xff)) - ((int)(b2[s2] & 0xff));
-      }
-      return (ret);
-    }
-  }
-}

+ 0 - 43
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/HiveObjectSerializer.java

@@ -1,43 +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.ql.io;
-
-import java.util.*;
-import java.io.*;
-
-import org.apache.hadoop.hive.ql.exec.HiveObject;
-import org.apache.hadoop.io.Writable;
-
-
-/**
- * A Serializer that is repeatedly invoked for Hive Objects that all share the
- * same schema.
- **/
-public interface HiveObjectSerializer <T extends Writable> {
-
-  public void initialize (Properties p);
-
-  public void serialize(HiveObject ho, DataOutput out) throws IOException;
-  public HiveObject deserialize(DataInput in)  throws IOException;
-
-  public int compare(byte [] b1, int s1, int l1, byte [] b2, int s2, int l2);
-
-  public long getReadErrorCount();
-  public long getWriteErrorCount();
-}

+ 0 - 220
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NaiiveJSONSerializer.java

@@ -1,220 +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.ql.io;
-
-import java.util.*;
-import java.io.*;
-
-import org.apache.hadoop.hive.ql.exec.HiveObject;
-import org.apache.hadoop.hive.ql.exec.CompositeHiveObject;
-import org.apache.hadoop.hive.ql.exec.PrimitiveHiveObject;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.utils.ByteStream;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * Serializes and deserializes Hive Objects as a delimited strings.
- *
- * Lots of code shamelessly copied from NaiiveSerializer. 
- **/
-public class NaiiveJSONSerializer implements HiveObjectSerializer {
-
-  List<SerDeField> topLevelFields;
-
-  int separator = Utilities.ctrlaCode;
-  int terminator = Utilities.newLineCode;
-
-  long writeErrorCount = 0, readErrorCount = 0;
-  ByteStream.Output bos = new ByteStream.Output ();
-  int width = -1;
-  ArrayList<String> slist = new ArrayList<String> ();
-  boolean isPrimitive, isTopLevel = true;
-
-  private void setSeparator (int separator) {
-    this.separator = separator;
-  }
-
-  private void setTerminator (int terminator) {
-    this.terminator = terminator;
-  }
-
-  private void setIsTopLevel(boolean value) {
-    isTopLevel = value;
-  }
-
-  public NaiiveJSONSerializer () {}
-
-  public void initialize (Properties p) {
-    String separator = p.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT);
-    if(separator != null) {
-      setSeparator(Integer.parseInt(separator));
-    }
-  }
-
-  public void serialize(HiveObject ho, DataOutput os) throws IOException {
-    try {
-      if(topLevelFields == null) {
-        try {
-          if(ho.isPrimitive()) {
-            topLevelFields = HiveObject.nlist;
-            isPrimitive = true;
-          } else {
-            topLevelFields = ho.getFields();
-            isPrimitive = false;
-
-            //System.err.println("Naiive: Hive Object has "+topLevelFields.size()+" fields");
-          }
-        } catch (HiveException e) {
-          throw new RuntimeException ("Cannot get Fields from HiveObject");
-        }
-      }
-
-      if(isPrimitive) {
-        os.write(ho.getJavaObject().toString().getBytes("UTF-8"));
-      } else {
-        boolean first = true;
-        int i = -1;
-        for(SerDeField onef: topLevelFields) {
-          i++;
-
-          if(!first) {
-            os.write(separator);
-          } else {
-            first = false;
-          }
-          HiveObject nho = ho.get(onef);
-          if(nho == null)
-            continue;
-
-          os.write(nho.toString().getBytes("UTF-8"));
-        }
-      }
-
-      if(isTopLevel) {
-        os.write(terminator);
-      }
-    } catch (HiveException e) {
-      writeErrorCount++;
-    }
-  }
-
-  private final static String NSTR = "";
-  private static enum streamStatus {EOF, TERMINATED, NORMAL}
-  public HiveObject deserialize (DataInput in) throws IOException {
-    boolean more = true;
-    CompositeHiveObject nr = null;
-    int entries = 0;
-
-    if(width != -1) {
-      nr = new CompositeHiveObject (width);
-    } else {
-      slist.clear();
-    }
-
-    do {
-      bos.reset();
-      streamStatus ss = readColumn(in, bos);
-      if((ss == streamStatus.EOF) ||
-         (ss == streamStatus.TERMINATED)) {
-        // read off entire row/file
-        more = false;
-      }
-
-      entries ++;
-      String col;
-      if(bos.getCount() > 0) {
-        col = new String(bos.getData(), 0, bos.getCount(), "UTF-8");
-      } else {
-        col = NSTR;
-      }
-
-      if(width == -1) {
-        slist.add(col);
-      } else {
-        if(entries <= width) {
-          try {
-            nr.addHiveObject(new PrimitiveHiveObject(col));
-          } catch (HiveException e) {
-            e.printStackTrace();
-            throw new IOException (e.getMessage());
-          }
-        }
-      }
-    } while (more);
-
-    if (width == -1) {
-      width = entries;
-      nr = new CompositeHiveObject (width);
-      for(String col: slist) {
-        try {
-          nr.addHiveObject(new PrimitiveHiveObject(col));
-        } catch (HiveException e) {
-          e.printStackTrace();
-          throw new IOException (e.getMessage());
-        }
-      }
-      return (nr);
-    }
-
-    if(width > entries) {
-      // skip and move on ..
-      readErrorCount++;
-      return null;
-    } else {
-      return nr;
-    }
-  }
-
-  public long getReadErrorCount() {
-    return readErrorCount;
-  }
-
-  public long getWriteErrorCount() {
-    return writeErrorCount;
-  }
-
-  private streamStatus readColumn(DataInput in, OutputStream out) throws IOException {
-    while (true) {
-      int b;
-      try {
-        b = (int)in.readByte();
-      } catch (EOFException e) {
-        return streamStatus.EOF;
-      }
-
-      if (b == terminator) {
-        return streamStatus.TERMINATED;
-      }
-
-      if (b == separator) {
-        return streamStatus.NORMAL;
-      }
-
-      out.write(b);
-    }
-    // Unreachable
-  }
-
-  public int compare(byte [] b1, int s1, int l1, byte [] b2, int s2, int l2) {
-    // Since all data is strings - we just use lexicographic ordering
-    return WritableComparator.compareBytes(b1, s1, l2, b2, s2, l2);
-  }
-}

+ 0 - 253
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NaiiveSerializer.java

@@ -1,253 +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.ql.io;
-
-import java.util.*;
-import java.io.*;
-
-import org.apache.hadoop.hive.ql.exec.HiveObject;
-import org.apache.hadoop.hive.ql.exec.CompositeHiveObject;
-import org.apache.hadoop.hive.ql.exec.PrimitiveHiveObject;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.utils.ByteStream;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * Serializes and deserializes Hive Objects as a delimited strings.
- **/
-public class NaiiveSerializer implements HiveObjectSerializer {
-
-  List<SerDeField> topLevelFields;
-  NaiiveSerializer [] topLevelSerializers;
-
-  int separator = Utilities.ctrlaCode;
-  int terminator = Utilities.newLineCode;
-  byte[] nullByteArray;
-
-  long writeErrorCount = 0, readErrorCount = 0;
-  ByteStream.Output bos = new ByteStream.Output ();
-  int width = -1;
-  ArrayList<String> slist = new ArrayList<String> ();
-  boolean isPrimitive, isTopLevel = true;
-
-  private void setSeparator (int separator) {
-    this.separator = separator;
-  }
-
-  private void setIsTopLevel(boolean value) {
-    isTopLevel = value;
-  }
-
-  private void setNullByteArray(byte[] nullByteArray) {
-    this.nullByteArray = nullByteArray;
-  }
-
-  public NaiiveSerializer () {
-    try {
-      setNullByteArray(Utilities.nullStringStorage.getBytes("UTF-8"));
-    } catch (UnsupportedEncodingException e) {
-      throw new RuntimeException("UTF-8 should be supported", e);
-    }
-  }
-
-  public void initialize (Properties p) {
-    String separator = p.getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT);
-    if(separator != null) {
-      setSeparator(Integer.parseInt(separator));
-    }
-    // Will make this configurable when DDL (CREATE TABLE) supports customized null string.
-    String nullString = null;
-    try {
-      if (nullString != null) {
-        setNullByteArray(nullString.getBytes("UTF-8"));
-      } else {
-        setNullByteArray(Utilities.nullStringStorage.getBytes("UTF-8"));
-      }
-    } catch (UnsupportedEncodingException e) {
-      throw new RuntimeException("UTF-8 should be supported", e);
-    }
-  }
-
-  public void serialize(HiveObject ho, DataOutput os) throws IOException {
-    try {
-      if (ho.getIsNull()) {
-        os.write(nullByteArray);
-      } else {
-        if(topLevelFields == null) {
-          try {
-            if(ho.isPrimitive()) {
-              topLevelFields = HiveObject.nlist;
-              isPrimitive = true;
-            } else {
-              topLevelFields = ho.getFields();
-              isPrimitive = false;
-              assert(topLevelFields != null);
-              topLevelSerializers = new NaiiveSerializer [topLevelFields.size()];
-              for(int i=0; i<topLevelFields.size(); i++) {
-                topLevelSerializers[i] = new NaiiveSerializer();
-                topLevelSerializers[i].setSeparator(separator+1);
-                topLevelSerializers[i].setIsTopLevel(false);
-                topLevelSerializers[i].setNullByteArray(nullByteArray);
-              }
-
-              //System.err.println("Naiive: Hive Object has "+topLevelFields.size()+" fields");
-            }
-          } catch (HiveException e) {
-            throw new RuntimeException ("Cannot get Fields from HiveObject", e);
-          }
-        }
-  
-        if(isPrimitive) {
-          os.write(ho.getJavaObject().toString().getBytes("UTF-8"));
-        } else {
-          boolean first = true;
-          int i = -1;
-          for(SerDeField onef: topLevelFields) {
-            i++;
-  
-            if(!first) {
-              os.write(separator);
-            } else {
-              first = false;
-            }
-            HiveObject nho = ho.get(onef);
-            if(nho == null)
-              continue;
-  
-            if(nho.isPrimitive()) {
-              os.write(nho.getJavaObject().toString().getBytes("UTF-8"));
-            } else {
-              topLevelSerializers[i].serialize(nho, os);
-            }
-          }
-        }
-      }
-      
-      if(isTopLevel) {
-        os.write(terminator);
-      }
-    } catch (HiveException e) {
-      writeErrorCount++;
-    }
-  }
-
-  private final static String NSTR = "";
-  private static enum streamStatus {EOF, TERMINATED, NORMAL}
-  public HiveObject deserialize (DataInput in) throws IOException {
-    boolean more = true;
-    CompositeHiveObject nr = null;
-    int entries = 0;
-
-    if(width != -1) {
-      nr = new CompositeHiveObject (width);
-    } else {
-      slist.clear();
-    }
-
-    do {
-      bos.reset();
-      streamStatus ss = readColumn(in, bos);
-      if((ss == streamStatus.EOF) ||
-         (ss == streamStatus.TERMINATED)) {
-        // read off entire row/file
-        more = false;
-      }
-
-      entries ++;
-      String col;
-      if(bos.getCount() > 0) {
-        col = new String(bos.getData(), 0, bos.getCount(), "UTF-8");
-      } else {
-        col = NSTR;
-      }
-
-      if(width == -1) {
-        slist.add(col);
-      } else {
-        if(entries <= width) {
-          try {
-            nr.addHiveObject(new PrimitiveHiveObject(col));
-          } catch (HiveException e) {
-            e.printStackTrace();
-            throw new IOException (e.getMessage());
-          }
-        }
-      }
-    } while (more);
-
-    if (width == -1) {
-      width = entries;
-      nr = new CompositeHiveObject (width);
-      for(String col: slist) {
-        try {
-          nr.addHiveObject(new PrimitiveHiveObject(col));
-        } catch (HiveException e) {
-          e.printStackTrace();
-          throw new IOException (e.getMessage());
-        }
-      }
-      return (nr);
-    }
-
-    if(width > entries) {
-      // skip and move on ..
-      readErrorCount++;
-      return null;
-    } else {
-      return nr;
-    }
-  }
-
-  public long getReadErrorCount() {
-    return readErrorCount;
-  }
-
-  public long getWriteErrorCount() {
-    return writeErrorCount;
-  }
-
-  private streamStatus readColumn(DataInput in, OutputStream out) throws IOException {
-    while (true) {
-      int b;
-      try {
-        b = (int)in.readByte();
-      } catch (EOFException e) {
-        return streamStatus.EOF;
-      }
-
-      if (b == terminator) {
-        return streamStatus.TERMINATED;
-      }
-
-      if (b == separator) {
-        return streamStatus.NORMAL;
-      }
-
-      out.write(b);
-    }
-    // Unreachable
-  }
-
-  public int compare(byte [] b1, int s1, int l1, byte [] b2, int s2, int l2) {
-    // Since all data is strings - we just use lexicographic ordering
-    return WritableComparator.compareBytes(b1, s1, l2, b2, s2, l2);
-  }
-}

+ 0 - 28
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NoTagHiveObjectComparator.java

@@ -1,28 +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.ql.io;
-
-public class NoTagHiveObjectComparator extends HiveObjectComparator {
-  /**
-   * Need to initialize base class with right key type
-   */
-  public NoTagHiveObjectComparator () {
-    super(NoTagWritableComparableHiveObject.class);
-  }
-}

+ 0 - 64
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NoTagWritableComparableHiveObject.java

@@ -1,64 +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.ql.io;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.exec.HiveObject;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.plan.mapredWork;
-
-/**
- * Extension of WritableComparableHiveObject that does not tag objects
- *
- */
-
-public class NoTagWritableComparableHiveObject extends WritableComparableHiveObject implements WritableComparable {
-
-  /**
-   * Constructor called by Hive on map output
-   */
-  public NoTagWritableComparableHiveObject (HiveObject ho,  HiveObjectSerializer hos) {
-    super(-1, ho, hos);
-  }
-
-  /**
-   * Default constructor invoked when map-reduce is constructing this object
-   */
-  public NoTagWritableComparableHiveObject () {
-    super();
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    ho = hos.deserialize(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    hos.serialize(ho, out);
-  }
-
-}

+ 0 - 82
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/NoTagWritableHiveObject.java

@@ -1,82 +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.ql.io;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.exec.HiveObject;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.plan.mapredWork;
-
-
-/**
- * A wrapper over hive objects that allows interfacing with Map-Reduce
- * serialization layer.
- *
- * NoTag Writable Hive Objects are deserialized only in reduce phase. They are
- * used only when the 'value' fields in the reduce phase are homogenous and don't 
- * require tagging
- * 
- */
-
-public class NoTagWritableHiveObject extends WritableHiveObject implements Writable, Configurable {
-
-  public NoTagWritableHiveObject () { super(); }
-
-  /**
-   * This constructor will be invoked by hive when creating writable  objects.
-   */
-  public NoTagWritableHiveObject (int tag, HiveObject ho,  HiveObjectSerializer hos) {
-    throw new RuntimeException ("NoTagWritables should not be initialized with tags");
-  }
-
-  public NoTagWritableHiveObject (HiveObject ho,  HiveObjectSerializer hos) {
-    super(-1, ho, hos);
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-    // we need just one deserializer. Get the first of them!
-    hos = mapredDeserializer[0];
-  }
-
-  @Override
-  public int getTag() {
-    return -1;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    // don't serialize tag
-    hos.serialize(ho, out);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    // don't de-serialize tag
-    ho = hos.deserialize(in);
-  }
-}

+ 0 - 206
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/WritableComparableHiveObject.java

@@ -1,206 +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.ql.io;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.exec.HiveObject;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.plan.mapredWork;
-
-/**
- * A wrapper over hive objects that allows interfacing with Map-Reduce
- * sorting/serialization layers. 
- *
- * WritableComparable Hive Objects are deserialized both in map and reduce phases.
- * Because they have a fixed schema - we just need to initialize a single deserializer
- * Either the serializer/deserializer is passed in at construction time (map output)
- * or it is obtained via JobConf at construction time
- *
- * The base version allows a tag to be serialized out alongside. But it is not used for
- * grouping/partitioning (only for sorting).
- *
- */
-
-public class WritableComparableHiveObject extends WritableHiveObject implements WritableComparable {
-
-  /**
-   * number of fields used in partition function.
-   * 0 - no fields will be used
-   * -1 - a random number will be used.
-   * Integer.MAX_VALUE - use all key fields.
-   */
-  static int numPartitionFields = Integer.MAX_VALUE;
-  static Random random = new Random();
-  
-  public static void setNumPartitionFields(int numPartitionFields) {
-    WritableComparableHiveObject.numPartitionFields = numPartitionFields;
-  }
-  
-  static List<SerDeField> topLevelFields = null;
-  static boolean isPrimitive;
-  static HiveObjectSerializer gHos;
-
-  /**
-   * Constructor called by Hive on map output
-   */
-  public WritableComparableHiveObject(int tag, HiveObject ho,  HiveObjectSerializer hos) {
-    super(tag, ho, hos);
-  }
-
-  /**
-   * Default constructor invoked when map-reduce is constructing this object
-   */
-  public WritableComparableHiveObject () {
-    super();
-  }
-
-  /**
-   * This function is invoked when map-reduce is constructing this object
-   * We construct one global deserializer for this case.
-   *
-   * TODO: how to do this initialization without making this configurable?
-   * Need to find a very early hook!
-   * 
-   * TODO: Replace NaiiveSerializer with MetadataTypedSerDe, and configure
-   * the MetadataTypedSerDe right here.
-   */
-  @Override
-  public void setConf(Configuration conf) {
-    if(gHos == null) {
-      mapredWork gWork = Utilities.getMapRedWork (conf);
-      gHos = new NaiiveSerializer();
-    }
-    hos = gHos;
-  }
-
-  /**
-   * Get tag out first like the base class - but use the same deserializer
-   * for getting the hive object (since the key schema is constant)
-   */
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    tag = (int) in.readByte();
-    ho = hos.deserialize(in);
-  }
-
-
-  public boolean equals (Object o) {
-    // afaik - this is never called
-    throw new RuntimeException("Not invented here");
-  }
-
-  /**
-   * This api should only be called during the reduce to check for group equality
-   * This asserts default grouping behavior (group by all columns). Note we don't
-   * use tags for grouping. Tags are only used for sorting and this behavior is
-   * controlled by HiveObjectComparator
-   * TODO (low priority): Make it possible to specify the grouping columns.
-   */
-  public int compareTo(Object o) {
-    HiveObject ho_rhs = ((WritableComparableHiveObject)o).getHo();
-    if(topLevelFields == null) {
-      try {
-        if(ho.isPrimitive()) {
-          topLevelFields = HiveObject.nlist;
-          isPrimitive = true;
-        } else {
-          topLevelFields = ho.getFields();
-          isPrimitive = false;
-        }
-      } catch (HiveException e) {
-        throw new RuntimeException ("Cannot get Fields from HiveObject");
-      }
-    }
-    try {
-
-      if(isPrimitive) {
-        Comparable a = (Comparable)ho.getJavaObject();
-        Comparable b = (Comparable)ho_rhs.getJavaObject();
-        return  a.compareTo(b);
-      }
-
-      // there is an assumption that the number of fields are the same.
-      // and that all the constituent fields are comparables.
-      // Assumption #1 has to be enforced by the deserializer.
-      // Assumption #2 has to be enforced by only allow primitive comparable types 
-      // as group fields.
-      for(SerDeField onef: topLevelFields) {
-        Comparable a = (Comparable)ho.get(onef).getJavaObject();
-        Comparable b = (Comparable)ho_rhs.get(onef).getJavaObject();
-        
-        int ret = a.compareTo(b);
-        if(ret != 0)
-          return (ret);
-      }
-    } catch (HiveException e) {
-      e.printStackTrace();
-      throw new RuntimeException ("HiveObject.get()/getJavaObject() methods failed");
-    }
-
-    // all fields are the same.
-    return (0);
-  }
-
-  public int hashCode() {
-    // This is a special case when we want the rows to be randomly distributed to  
-    // reducers for load balancing problem.  In this case, we use a random number 
-    // as the hashCode.
-    if (numPartitionFields == -1) {
-      return random.nextInt();
-    }
-    if(topLevelFields == null) {
-      try {
-        if(ho.isPrimitive()) {
-          topLevelFields = HiveObject.nlist;
-          isPrimitive = true;
-        } else {
-          topLevelFields = ho.getFields();
-          isPrimitive = false;
-        }
-      } catch (HiveException e) {
-        throw new RuntimeException ("Cannot get Fields from HiveObject");
-      }
-    }
-
-    int ret = 0;
-    try {
-      if(isPrimitive) {
-        return ho.getJavaObject().hashCode();
-      }
-      int numFields = 0;
-      for(SerDeField onef: topLevelFields) {
-        Object o = ho.get(onef).getJavaObject();
-        // TODO: replace with something smarter (borrowed from Text.java)
-        ret = ret * 31 + (o == null ? 0 : o.hashCode());
-        numFields ++;
-        if (numFields >= numPartitionFields) break;
-      }
-    } catch (HiveException e) {
-      e.printStackTrace();
-      throw new RuntimeException ("HiveObject.get()/getJavaObject() failed");
-    }
-    return (ret);
-  }
-}

+ 0 - 149
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/io/WritableHiveObject.java

@@ -1,149 +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.ql.io;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.serde.*;
-import org.apache.hadoop.hive.ql.exec.HiveObject;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.plan.mapredWork;
-
-
-/**
- * A wrapper over hive objects that allows interfacing with Map-Reduce
- * serialization layer.
- *
- * Writable Hive Objects are deserialized only in reduce phase. 'Value'
- * fields are encapsulated in WritableHiveObjects - and they are heterogenous.
- * Hence we use prefix a tag to the actual object to figure out different 
- * kinds of objects
- * 
- */
-
-public class WritableHiveObject implements Writable, Configurable {
-  protected int tag;
-  protected HiveObject ho;
-  protected HiveObjectSerializer hos;
-
-  protected static HiveObjectSerializer [] mapredDeserializer;
-
-
-  public WritableHiveObject () {}
-
-  /**
-   * Should not be called.
-   */
-  public Configuration getConf() {
-    throw new RuntimeException ("Unexpected invocation");
-  }
-
-  /**
-   * 
-   * 1. we come through this code path when being initialized by map-reduce
-   * 
-   *    In this case we fall back on one common deserializer - but create an
-   *    instance for each tag (since deserializers cannot deal with differing
-   *    schemas
-   *
-   *    It turns out that objects may be deserialized and again serialized. This
-   *    is done unnecessarily in some code paths in 0.15 - but is legitimate in 
-   *    in the case of combiner.
-   *
-   * TODO: how to do this initialization without making this configurable? Need to 
-   * find a very early hook!
-   */
-  public void setConf(Configuration conf) {
-    if(mapredDeserializer == null) {
-      mapredWork gWork = Utilities.getMapRedWork (conf);
-      setSerialFormat();
-    }
-  }
-  
-  /**
-   * Meant to be accessed directly from test code only 
-   * 
-   * TODO: this deserializers need to be initialized with the schema
-   */
-  public static void setSerialFormat() {
-    mapredDeserializer = 
-      new HiveObjectSerializer [(int) Byte.MAX_VALUE];
-    
-    for(int i=0; i<Byte.MAX_VALUE; i++) {
-      // we initialize a deserializer for each tag. in future we will
-      // pass the schema in the deserializer as well. For now ..
-      // TODO: use MetadataTypedSerDe to replace NaviiveSerializer.
-      mapredDeserializer[i] =  new NaiiveSerializer();
-    }
-  }
-
-
-  /**
-   * 2. when map-reduce initializes the object - we just read data.
-   *    for each row we read the tag and then use the deserializer for
-   *    that tag.
-   *
-   */
-  public void readFields(DataInput in) throws IOException {
-    tag = (int) in.readByte();
-    // stash away the serializer in case we are written out
-    hos = mapredDeserializer[tag];
-    ho = hos.deserialize(in);
-  }
-
-  /**
-   * 1. this constructor will be invoked by hive when creating writable  objects
-   */
-  public WritableHiveObject (int tag, HiveObject ho,  HiveObjectSerializer hos) {
-    this.tag = tag;
-    this.ho = ho;
-    this.hos = hos;
-  }
-
-  /**
-   * 2. when Hive instantiates Writable objects - we will repeatedly set a new object
-   */
-  public void setHo(HiveObject ho) {
-    this.ho = ho;
-  }
-
-  /**
-   * 3. and ask for the object to be serialized out
-   */
-  public void write(DataOutput out) throws IOException {
-    out.write(tag);
-    hos.serialize(ho, out);
-  }
-
-
-  public HiveObject getHo() {
-    return (ho);
-  }
-
-  public int getTag() {
-    return tag;
-  }
-}
-
-

+ 24 - 10
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Constants;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -158,7 +157,7 @@ public class Hive {
     tbl.setOutputFormatClass(fileOutputFormat.getName());
 
     for (String col: columns) {
-      FieldSchema field = new FieldSchema(col, Constants.STRING_TYPE_NAME, "default string type");
+      FieldSchema field = new FieldSchema(col, org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "default");
       tbl.getCols().add(field);
     }
 
@@ -166,11 +165,11 @@ public class Hive {
       for (String partCol : partCols) {
         FieldSchema part = new FieldSchema();
         part.setName(partCol);
-        part.setType(Constants.STRING_TYPE_NAME); // default partition key
+        part.setType(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME); // default partition key
         tbl.getPartCols().add(part);
       }
     }
-    tbl.setSerializationLib(org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe.shortName());
+    tbl.setSerializationLib(org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe.shortName());
     tbl.setNumBuckets(bucketCount);
     createTable(tbl);
   }
@@ -281,6 +280,8 @@ public class Hive {
     try {
       // first get a schema (in key / vals)
       Properties p = MetaStoreUtils.getSchema(tTable);
+      // Map hive1 to hive3 class names, can be removed when migration is done.
+      p = MetaStoreUtils.hive1Tohive3ClassNames(p);
       table.setSchema(p);
       table.setTTable(tTable);
       table.setInputFormatClass((Class<? extends InputFormat<WritableComparable, Writable>>)
@@ -289,16 +290,18 @@ public class Hive {
       table.setOutputFormatClass((Class<? extends OutputFormat<WritableComparable, Writable>>)
           Class.forName(table.getSchema().getProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT,
               org.apache.hadoop.mapred.SequenceFileOutputFormat.class.getName()))); 
-      table.setSerDe(MetaStoreUtils.getSerDe(getConf(), p));
+      table.setDeserializer(MetaStoreUtils.getDeserializer(getConf(), p));
       table.setDataLocation(new URI(tTable.getSd().getLocation()));
     } catch(Exception e) {
       LOG.error(StringUtils.stringifyException(e));
       throw new HiveException(e);
     }
-    String sf = table.getSerializationFormat();
-    char[] b = sf.toCharArray();
-    if ((b.length == 1) && (b[0] < 10)){ // ^A, ^B, ^C, ^D, \t
-      table.setSerializationFormat(Integer.toString(b[0]));
+    String sf = table.getSerdeParam(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT);
+    if(sf != null) {
+      char[] b = sf.toCharArray();
+      if ((b.length == 1) && (b[0] < 10)){ // ^A, ^B, ^C, ^D, \t
+        table.setSerdeParam(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, Integer.toString(b[0]));
+      }
     }
     table.checkValidity();
     return table;
@@ -459,6 +462,17 @@ public class Hive {
     }
     return new Partition(tbl, tpart);
   }
+  
+  public List<String> getPartitionNames(String dbName, String tblName, short max) throws HiveException {
+    List names = null;
+    try {
+      names = msc.listPartitionNames(dbName, tblName, max);
+    } catch (Exception e) {
+      LOG.error(StringUtils.stringifyException(e));
+      throw new HiveException(e);
+    }
+    return names;
+  }
 
   /**
    * get all the partitions that the table has
@@ -494,7 +508,7 @@ public class Hive {
       // create an empty partition. 
       // HACK, HACK. SemanticAnalyzer code requires that an empty partition when the table is not partitioned
       org.apache.hadoop.hive.metastore.api.Partition tPart = new org.apache.hadoop.hive.metastore.api.Partition();
-      tPart.setSd(tbl.getTTable().getSd());
+      tPart.setSd(tbl.getTTable().getSd()); // TODO: get a copy
       Partition part = new Partition(tbl, tPart);
       ArrayList<Partition> parts = new ArrayList<Partition>(1);
       parts.add(part);

+ 0 - 132
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java

@@ -18,13 +18,6 @@
 
 package org.apache.hadoop.hive.ql.metadata;
 
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-
-import org.apache.hadoop.hive.serde.*;
-
 /**
  * General collection of helper functions
  *
@@ -38,131 +31,6 @@ public class HiveUtils {
   public static final String LBRACE = "{";
   public static final String RBRACE = "}";
 
-  /**
-   * Common functionality for all SerDe libraries to handle list and primitive
-   * string serialization to json strings (who uses this?)
-   */
-  public static String toJSONString(Object obj, SerDeField hf, SerDe sd) throws HiveException {
-
-    if(hf.isList()) {
-      Class type = hf.getListElementType();
-      boolean is_string = (type == String.class);
-      boolean is_boolean = (type == Boolean.class);
-      boolean is_primitive = ReflectionSerDeField.isClassPrimitive(type);
-
-      Iterator iter = ((List)obj).iterator();
-      StringBuilder sb = new StringBuilder(LBRACKET);
-      String toPrefix = "";
-      boolean first = true;
-
-      while(iter.hasNext()) {
-        Object lobj = iter.next();
-
-        if(is_primitive) {
-          if(is_string) {
-            sb.append(toPrefix);
-            sb.append(QUOTE);
-            sb.append(escapeString((String)lobj));
-            sb.append(QUOTE);
-          } else if(is_boolean) {
-            sb.append(toPrefix+(((Boolean)lobj).booleanValue() ? "True" : "False"));
-          } else {
-            // it's a number - so doesn't need to be escaped.
-            sb.append(toPrefix+lobj.toString());
-          }
-        } else {
-          try {
-            sb.append(toPrefix+sd.toJSONString(lobj, null));
-          } catch(SerDeException e) {
-            throw new HiveException(e);
-          }
-        }
-
-        if(first) {
-          toPrefix = ",";
-          first = false;
-        }
-      }
-      sb.append(RBRACKET);
-      return (sb.toString());
-    } else if (hf.isMap()) {
-      Class keyType = hf.getMapKeyType();
-      Class valType = hf.getMapValueType();
-
-      boolean key_is_string = (keyType == String.class);
-      boolean key_is_boolean = (keyType == Boolean.class);
-      boolean key_is_primitive = ReflectionSerDeField.isClassPrimitive(keyType);
-
-      boolean val_is_string = (valType == String.class);
-      boolean val_is_boolean = (valType == Boolean.class);
-      boolean val_is_primitive = ReflectionSerDeField.isClassPrimitive(valType);
-
-
-
-      Iterator iter = ((Map)obj).keySet().iterator();
-      StringBuilder sb = new StringBuilder(LBRACE);
-      String toPrefix = "";
-      boolean first = true;
-
-      while(iter.hasNext()) {
-        Object lobj = iter.next();
-        Object robj = ((Map)obj).get(lobj);
-
-        // Emit key
-        if(key_is_primitive) {
-          if(key_is_string) {
-            sb.append(toPrefix);
-            sb.append(QUOTE);
-            sb.append(escapeString((String)lobj));
-            sb.append(QUOTE);
-          } else if(key_is_boolean) {
-            sb.append(toPrefix+(((Boolean)lobj).booleanValue() ? "True" : "False"));
-          } else {
-            // it's a number - so doesn't need to be escaped.
-            sb.append(toPrefix+lobj.toString());
-          }
-        } else {
-          try {
-            sb.append(toPrefix+sd.toJSONString(lobj, null));
-          } catch(SerDeException e) {
-            throw new HiveException(e);
-          }
-        }
-
-        sb.append(COLON);
-
-        // Emit val
-        if(val_is_primitive) {
-          if(val_is_string) {
-            sb.append(toPrefix);
-            sb.append(QUOTE);
-            sb.append(escapeString((String)robj));
-            sb.append(QUOTE);
-          } else if(val_is_boolean) {
-            sb.append(toPrefix+(((Boolean)robj).booleanValue() ? "True" : "False"));
-          } else {
-            // it's a number - so doesn't need to be escaped.
-            sb.append(toPrefix+robj.toString());
-          }
-        } else {
-          try {
-            sb.append(toPrefix+sd.toJSONString(robj, null));
-          } catch(SerDeException e) {
-            throw new HiveException(e);
-          }
-        }
-
-        if(first) {
-          toPrefix = ",";
-          first = false;
-        }
-      }
-      sb.append(RBRACE);
-      return (sb.toString());
-    } else {
-      throw new HiveException("HiveUtils.toJSONString only does lists");
-    }
-  }
 
   public static String escapeString(String str) {
     int length = str.length();

+ 28 - 5
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.regex.Matcher;
@@ -35,6 +36,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 
 /**
  * A Hive Table Partition: is a fundamental storage unit within a Table
@@ -46,6 +49,13 @@ public class Partition {
 
     private Table table;
     private org.apache.hadoop.hive.metastore.api.Partition tPartition;
+    /**
+     * @return the tPartition
+     */
+    public org.apache.hadoop.hive.metastore.api.Partition getTPartition() {
+      return tPartition;
+    }
+
     private LinkedHashMap<String, String> spec;
     
     private Path partPath;
@@ -54,7 +64,7 @@ public class Partition {
     Partition(Table tbl, org.apache.hadoop.hive.metastore.api.Partition tp) throws HiveException {
       this.table = tbl;
       this.tPartition = tp;
-      String partName = "";
+      partName = "";
       if(table.isPartitioned()) {
         try {
           partName = Warehouse.makePartName(tbl.getPartCols(), tp.getValues());
@@ -83,7 +93,14 @@ public class Partition {
       this.table = tbl;
       // initialize the tPartition(thrift object) with the data from path and  table
       this.tPartition = new org.apache.hadoop.hive.metastore.api.Partition();
-      this.tPartition.setSd(tbl.getTTable().getSd());
+      this.tPartition.setDbName(tbl.getDbName());
+      this.tPartition.setTableName(tbl.getName());
+      StorageDescriptor sd = tbl.getTTable().getSd();
+      StorageDescriptor psd = new StorageDescriptor(
+          sd.getCols(), sd.getLocation(), sd.getInputFormat(), sd.getOutputFormat(),
+          sd.isCompressed(), sd.getNumBuckets(), sd.getSerdeInfo(), sd.getBucketCols(),
+          sd.getSortCols(), new HashMap<String, String>());
+      this.tPartition.setSd(psd);
       // change the partition location
       if(table.isPartitioned()) {
         this.partPath = path;
@@ -93,12 +110,17 @@ public class Partition {
         this.partPath = table.getPath();
       }
       spec = makeSpecFromPath();
-      tPartition.getSd().setLocation(partPath.toString());
+      psd.setLocation(this.partPath.toString());
       List<String> partVals = new ArrayList<String> ();
       tPartition.setValues(partVals);
       for (FieldSchema field : tbl.getPartCols()) {
         partVals.add(spec.get(field.getName()));
       }
+      try {
+        this.partName = Warehouse.makePartName(tbl.getPartCols(), partVals);
+      } catch (MetaException e) {
+        throw new HiveException("Invalid partition key values", e);
+      }
     }
     
     static final Pattern pat = Pattern.compile("([^/]+)=([^/]+)");
@@ -162,8 +184,7 @@ public class Partition {
     }
 
     public String getName() {
-        // starting to look really ugly now
-        return getPath()[0].toString();
+        return partName;
     }
 
     public Table getTable() {
@@ -242,6 +263,8 @@ public class Partition {
      * mapping from a Path to the bucket number if any
      */
     private static Pattern bpattern = Pattern.compile("part-([0-9][0-9][0-9][0-9][0-9])");
+
+    private String partName;
     @SuppressWarnings("nls")
     public static int getBucketNum(Path p) {
         Matcher m = bpattern.matcher(p.getName());

+ 114 - 157
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java

@@ -40,8 +40,13 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.serde.SerDe;
-import org.apache.hadoop.hive.serde.SerDeField;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.InputFormat;
@@ -57,7 +62,7 @@ public class Table {
   static final private Log LOG = LogFactory.getLog("hive.ql.metadata.Table");
 
   private Properties schema;
-  private SerDe serDe;
+  private Deserializer deserializer;
   private URI uri;
   private Class<? extends InputFormat> inputFormatClass;
   private Class<? extends OutputFormat> outputFormatClass;
@@ -83,73 +88,77 @@ public class Table {
    *
    * @exception HiveException on internal error. Note not possible now, but in the future reserve the right to throw an exception
    */
-  public Table(String name, Properties schema, SerDe serDe, 
+  public Table(String name, Properties schema, Deserializer deserializer, 
       Class<? extends InputFormat<?, ?>> inputFormatClass,
       Class<? extends OutputFormat<?, ?>> outputFormatClass,
       URI dataLocation, Hive hive) throws HiveException {
     initEmpty();
-    this.getTTable().setTableName(name);
-    this.getTTable().getSd().setLocation(dataLocation.toASCIIString());
-    this.setInputFormatClass(inputFormatClass);
-    this.setOutputFormatClass(outputFormatClass);
-    this.setDataLocation(dataLocation);
     this.schema = schema;
-    this.serDe = serDe; //TODO: convert to SerDeInfo format
-    this.getTTable().getSd().getSerdeInfo().setSerializationLib(serDe.getClass().getName());
+    this.deserializer = deserializer; //TODO: convert to SerDeInfo format
+    this.getTTable().getSd().getSerdeInfo().setSerializationLib(deserializer.getShortName());
+    getTTable().setTableName(name);
+    getSerdeInfo().setSerializationLib(deserializer.getClass().getName());
+    setInputFormatClass(inputFormatClass);
+    setOutputFormatClass(outputFormatClass);
+    setDataLocation(dataLocation);
   }
   
   public Table(String name) {
     // fill in defaults
     initEmpty();
-    this.getTTable().setTableName(name);
-    this.getTTable().setDatabase(MetaStoreUtils.DEFAULT_DATABASE_NAME);
-    this.getTTable().getSd().getSerdeInfo().setSerializationLib(org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe.class.getName());
-    this.getTTable().getSd().getSerdeInfo().setSerializationFormat("1");
+    getTTable().setTableName(name);
+    getTTable().setDbName(MetaStoreUtils.DEFAULT_DATABASE_NAME);
+    getSerdeInfo().setSerializationLib(MetadataTypedColumnsetSerDe.shortName());
+    getSerdeInfo().getParameters().put(Constants.SERIALIZATION_FORMAT, "1");
   }
   
   void initEmpty() {
-    this.setTTable(new org.apache.hadoop.hive.metastore.api.Table());
-    this.getTTable().setSd(new StorageDescriptor());
-    this.getTTable().getSd().setSerdeInfo(new SerDeInfo());
-    this.getTTable().getSd().setNumBuckets(-1);
-    this.getTTable().getSd().setBucketCols(new ArrayList<String>());
-    this.getTTable().getSd().setCols(new ArrayList<FieldSchema>());
-    this.getTTable().setPartitionKeys(new ArrayList<FieldSchema>());
-    this.getTTable().setParameters(new HashMap<String, String>());
-    this.getTTable().getSd().setParameters(new HashMap<String, String>());
-    this.getTTable().getSd().setSortCols(new ArrayList<Order>());
+    setTTable(new org.apache.hadoop.hive.metastore.api.Table());
+    getTTable().setSd(new StorageDescriptor());
+    getTTable().setPartitionKeys(new ArrayList<FieldSchema>());
+    getTTable().setParameters(new HashMap<String, String>());
+
+    StorageDescriptor sd = getTTable().getSd();
+    sd.setSerdeInfo(new SerDeInfo());
+    sd.setNumBuckets(-1);
+    sd.setBucketCols(new ArrayList<String>());
+    sd.setCols(new ArrayList<FieldSchema>());
+    sd.setParameters(new HashMap<String, String>());
+    sd.setSortCols(new ArrayList<Order>());
+    
+    sd.getSerdeInfo().setParameters(new HashMap<String, String>());
   }
   
   protected void initSerDe() throws HiveException {
-    if(this.serDe == null) {
+    if (deserializer == null) {
       try {
-        this.serDe = MetaStoreUtils.getSerDe(Hive.get().getConf(), this.getTTable());
+        deserializer = MetaStoreUtils.getDeserializer(Hive.get().getConf(), this.getTTable());
       } catch (MetaException e) {
         throw new HiveException(e);
       }
     }
   }
-  public void checkValidity() throws HiveException {
 
+  public void checkValidity() throws HiveException {
     // check for validity
-    String name = this.getTTable().getTableName();
+    String name = getTTable().getTableName();
     if (null == name || name.length() == 0 || !MetaStoreUtils.validateName(name)) {
       throw new HiveException("[" + name + "]: is not a valid table name");
     }
-    if (null == this.getSerDe()) {
+    if (null == getDeserializer()) {
       throw new HiveException("must specify a non-null serDe");
     }
-    if (null == this.getInputFormatClass()) {
+    if (null == getInputFormatClass()) {
       throw new HiveException("must specify an InputFormat class");
     }
-    if (null == this.getOutputFormatClass()) {
+    if (null == getOutputFormatClass()) {
       throw new HiveException("must specify an OutputFormat class");
     }
     return;
   }
 
   /**
-   * @param inputFormatClass the inputFormatClass to set
+   * @param inputFormatClass 
    */
   public void setInputFormatClass(Class<? extends InputFormat> inputFormatClass) {
     this.inputFormatClass = inputFormatClass;
@@ -157,7 +166,7 @@ public class Table {
   }
 
   /**
-   * @param outputFormatClass the outputFormatClass to set
+   * @param outputFormatClass 
    */
   public void setOutputFormatClass(Class<? extends OutputFormat> outputFormatClass) {
     this.outputFormatClass = outputFormatClass;
@@ -165,37 +174,37 @@ public class Table {
   }
 
   final public Properties getSchema()  {
-    return this.schema;
+    return schema;
   }
 
   final public Path getPath() {
-    return new Path(this.getTTable().getSd().getLocation());
+    return new Path(getTTable().getSd().getLocation());
   }
 
   final public String getName() {
-    return this.getTTable().getTableName();
+    return getTTable().getTableName();
   }
 
   final public URI getDataLocation() {
-    return this.uri;
+    return uri;
   }
 
-  final public SerDe getSerDe() {
-    return this.serDe;
+  final public Deserializer getDeserializer() {
+    return deserializer;
   }
 
   final public Class<? extends InputFormat> getInputFormatClass() {
-    return this.inputFormatClass;
+    return inputFormatClass;
   }
 
   final public Class<? extends OutputFormat> getOutputFormatClass() {
-    return this.outputFormatClass;
+    return outputFormatClass;
   }
 
   final public boolean isValidSpec(AbstractMap<String, String> spec) throws HiveException {
 
     // TODO - types need to be checked.
-    List<FieldSchema> partCols = this.getTTable().getPartitionKeys();
+    List<FieldSchema> partCols = getTTable().getPartitionKeys();
     if(partCols== null || (partCols.size() == 0)) {
       if (spec != null)
         throw new HiveException("table is not partitioned but partition spec exists: " + spec);
@@ -217,7 +226,7 @@ public class Table {
   }
   
   public void setProperty(String name, String value) {
-    this.getTTable().getParameters().put(name, value);
+    getTTable().getParameters().put(name, value);
   }
 
   /**
@@ -225,36 +234,31 @@ public class Table {
    *
    */
   public String getProperty(String name) {
-    return this.getTTable().getParameters().get(name);
+    return getTTable().getParameters().get(name);
   }
 
-  public Vector<SerDeField> getFields(String [] components) {
+  public Vector<StructField> getFields() {
 
-
-    Vector<SerDeField> fields = new Vector<SerDeField> ();
+    Vector<StructField> fields = new Vector<StructField> ();
     try {
-      SerDe decoder = getSerDe();
-      if (components == null || components.length == 0) {
-        // Expand out all the columns of the table
-        List<SerDeField> fld_lst = decoder.getFields(null);
-        for(SerDeField field: fld_lst) {
-          fields.add(field);
-        }
-      }
-      else {
-        for (int i = 0; i < components.length; i++) {
-          fields.add(decoder.getFieldFromExpression(null, components[i]));
-        }
+      Deserializer decoder = getDeserializer();
+
+      // Expand out all the columns of the table
+      StructObjectInspector structObjectInspector = (StructObjectInspector)decoder.getObjectInspector();
+      List<? extends StructField> fld_lst = structObjectInspector.getAllStructFieldRefs();
+      for(StructField field: fld_lst) {
+        fields.add(field);
       }
-    } catch (Exception e) {
+    } catch (SerDeException e) {
       throw new RuntimeException(e);
     }
     return fields;
   }
 
-  public SerDeField getField(String fld) {
+  public StructField getField(String fld) {
     try {
-      return getSerDe().getFieldFromExpression(null, fld);
+      StructObjectInspector structObjectInspector = (StructObjectInspector)getDeserializer().getObjectInspector();
+      return structObjectInspector.getStructFieldRef(fld);
     }
     catch (Exception e) {
       throw new RuntimeException(e);
@@ -271,25 +275,25 @@ public class Table {
   /**
    * @param serDe the serDe to set
    */
-  public void setSerDe(SerDe serDe) {
-    this.serDe = serDe;
+  public void setDeserializer(Deserializer deserializer) {
+    this.deserializer = deserializer;
   }
 
   public String toString() { 
-    return this.getTTable().getTableName();
+    return getTTable().getTableName();
   }
 
   public List<FieldSchema> getPartCols() {
-    List<FieldSchema> partKeys = this.getTTable().getPartitionKeys();
+    List<FieldSchema> partKeys = getTTable().getPartitionKeys();
     if(partKeys == null) {
       partKeys = new ArrayList<FieldSchema>();
-      this.getTTable().setPartitionKeys(partKeys);
+      getTTable().setPartitionKeys(partKeys);
     }
     return partKeys;
   }
   
   public boolean isPartitionKey(String colName) {
-    for (FieldSchema key : this.getPartCols()) {
+    for (FieldSchema key : getPartCols()) {
       if(key.getName().toLowerCase().equals(colName)) {
         return true;
       }
@@ -299,7 +303,7 @@ public class Table {
 
   //TODO merge this with getBucketCols function
   public String getBucketingDimensionId() {
-    List<String> bcols = this.getTTable().getSd().getBucketCols();
+    List<String> bcols = getTTable().getSd().getBucketCols();
     if(bcols == null || bcols.size() == 0) {
       return null;
     }
@@ -311,14 +315,6 @@ public class Table {
     return bcols.get(0);
   }
 
-  public String getSerializationFormat() {
-    return this.getTTable().getSd().getSerdeInfo().getSerializationFormat();
-  }
-
-  public void setSerializationFormat(String f) {
-    this.getTTable().getSd().getSerdeInfo().setSerializationFormat(f);
-  }
-
   /**
    * @return the tTable
    */
@@ -334,16 +330,8 @@ public class Table {
   }
 
   public void setDataLocation(URI uri2) {
-    this.uri = uri2;
-    this.getTTable().getSd().setLocation(uri2.toString());
-  }
-
-  public void setSerializationClass(String cls) {
-    this.getTTable().getSd().getSerdeInfo().setSerializationClass(cls);
-  }
-
-  public void setSerializationLib(String lib) {
-    this.getTTable().getSd().getSerdeInfo().setSerializationLib(lib);
+    uri = uri2;
+    getTTable().getSd().setLocation(uri2.toString());
   }
 
   public void setBucketCols(List<String> bucketCols) throws HiveException {
@@ -352,22 +340,18 @@ public class Table {
     }
 
     for (String col : bucketCols) {
-      if(!this.isField(col))
+      if(!isField(col))
         throw new HiveException("Bucket columns " + col + " is not part of the table columns" ); 
     }
-    this.getTTable().getSd().setBucketCols(bucketCols);
+    getTTable().getSd().setBucketCols(bucketCols);
   }
 
-  public void setSortCols(List<String> sortCols) throws HiveException {
-    List<Order> sortOrder = new ArrayList<Order>();
-    for (String col : sortCols) {
-      sortOrder.add(new Order(col, 1));
-    }
-    this.getTTable().getSd().setSortCols(sortOrder);
+  public void setSortCols(List<Order> sortOrder) throws HiveException {
+    getTTable().getSd().setSortCols(sortOrder);
   }
 
   private boolean isField(String col) {
-    for (FieldSchema field : this.getCols()) {
+    for (FieldSchema field : getCols()) {
       if(field.getName().equals(col)) {
         return true;
       }
@@ -376,31 +360,30 @@ public class Table {
   }
 
   public List<FieldSchema> getCols() {
-    return this.getTTable().getSd().getCols();
+    return getTTable().getSd().getCols();
   }
 
   public void setPartCols(List<FieldSchema> partCols) {
-    this.getTTable().setPartitionKeys(partCols);
+    getTTable().setPartitionKeys(partCols);
   }
 
   public String getDbName() {
-    return this.getTTable().getDatabase();
+    return getTTable().getDbName();
   }
 
   public int getNumBuckets() {
-    return this.getTTable().getSd().getNumBuckets();
+    return getTTable().getSd().getNumBuckets();
   }
   
   /**
-   * Replaces files in the partition with new data set specifed by srcf. Works by moving files
-   *
-   * @param srcf Files to be moved. Leaf Directories or Globbed File Paths
+   * Replaces files in the partition with new data set specified by srcf. Works by moving files
+   * @param srcf Files to be replaced. Leaf directories or globbed file paths
    */
   protected void replaceFiles(Path srcf) throws HiveException {
     FileSystem fs;
     try {
-      fs = FileSystem.get(this.getDataLocation(), Hive.get().getConf());
-      Hive.get().replaceFiles(srcf, new Path(this.getDataLocation().getPath()), fs);
+      fs = FileSystem.get(getDataLocation(), Hive.get().getConf());
+      Hive.get().replaceFiles(srcf, new Path(getDataLocation().getPath()), fs);
     } catch (IOException e) {
       throw new HiveException("addFiles: filesystem error in check phase", e);
     }
@@ -408,14 +391,13 @@ public class Table {
 
   /**
    * Inserts files specified into the partition. Works by moving files
-   *
-   * @param srcf Files to be moved. Leaf Directories or Globbed File Paths
+   * @param srcf Files to be moved. Leaf directories or globbed file paths
    */
   protected void copyFiles(Path srcf) throws HiveException {
     FileSystem fs;
     try {
-      fs = FileSystem.get(this.getDataLocation(), Hive.get().getConf());
-      Hive.get().copyFiles(srcf, new Path(this.getDataLocation().getPath()), fs);
+      fs = FileSystem.get(getDataLocation(), Hive.get().getConf());
+      Hive.get().copyFiles(srcf, new Path(getDataLocation().getPath()), fs);
     } catch (IOException e) {
       throw new HiveException("addFiles: filesystem error in check phase", e);
     }
@@ -423,7 +405,7 @@ public class Table {
 
   public void setInputFormatClass(String name) throws HiveException {
     try {
-      this.setInputFormatClass((Class<? extends InputFormat<WritableComparable, Writable>>)Class.forName(name));
+      setInputFormatClass((Class<? extends InputFormat<WritableComparable, Writable>>)Class.forName(name));
     } catch (ClassNotFoundException e) {
       throw new HiveException("Class not found: " + name, e);
     }
@@ -431,7 +413,7 @@ public class Table {
 
   public void setOutputFormatClass(String name) throws HiveException {
     try {
-      this.setOutputFormatClass((Class<? extends OutputFormat<WritableComparable, Writable>>)Class.forName(name));
+      setOutputFormatClass((Class<? extends OutputFormat<WritableComparable, Writable>>)Class.forName(name));
     } catch (ClassNotFoundException e) {
       throw new HiveException("Class not found: " + name, e);
     }
@@ -439,18 +421,18 @@ public class Table {
 
   
   public boolean isPartitioned() {
-    if(this.getPartCols() == null) {
+    if(getPartCols() == null) {
       return false;
     }
-    return (this.getPartCols().size() != 0);
+    return (getPartCols().size() != 0);
   }
 
   public void setFields(List<FieldSchema> fields) {
-    this.getTTable().getSd().setCols(fields);
+    getTTable().getSd().setCols(fields);
   }
 
   public void setNumBuckets(int nb) {
-    this.getTTable().getSd().setNumBuckets(nb);
+    getTTable().getSd().setNumBuckets(nb);
   }
 
   /**
@@ -493,56 +475,32 @@ public class Table {
     tTable.setRetention(retention);
   }
 
-  public String getSerializationLib() {
-    return this.getTTable().getSd().getSerdeInfo().getSerializationLib();
-  }
-
-  public String getSerializationClass() {
-    return this.getTTable().getSd().getSerdeInfo().getSerializationClass();
-  }
-
-  public void setIsCompressed(boolean b) {
-    this.getTTable().getSd().setIsCompressed(b);
-  }
-
-  public void setFieldDelim(String string) {
-    this.getTTable().getSd().getSerdeInfo().setFieldDelim(string);  
+  private SerDeInfo getSerdeInfo() {
+    return getTTable().getSd().getSerdeInfo();
   }
 
-  public void setCollectionItemDelim(String string) {
-    this.getTTable().getSd().getSerdeInfo().setCollectionItemDelim(string);  
-  }
-
-  public void setLineDelim(String string) {
-    this.getTTable().getSd().getSerdeInfo().setLineDelim(string);  
-  }
-
-  public void setMapKeyDelim(String string) {
-    this.getTTable().getSd().getSerdeInfo().setMapKeyDelim(string);  
-  }
-  
-  public String getFieldDelim() {
-    return this.getTTable().getSd().getSerdeInfo().getFieldDelim();
+  public void setSerializationLib(String lib) {
+    getSerdeInfo().setSerializationLib(lib);
   }
 
-  public String getCollectionItemDelim() {
-    return this.getTTable().getSd().getSerdeInfo().getCollectionItemDelim();
+  public String getSerializationLib() {
+    return getSerdeInfo().getSerializationLib();
   }
   
-  public String getLineDelim() {
-    return this.getTTable().getSd().getSerdeInfo().getLineDelim();
+  public String getSerdeParam(String param) {
+    return getSerdeInfo().getParameters().get(param);
   }
   
-  public String getMapKeyDelim() {
-    return this.getTTable().getSd().getSerdeInfo().getMapKeyDelim();
+  public String setSerdeParam(String param, String value) {
+    return getSerdeInfo().getParameters().put(param, value);
   }
 
   public List<String> getBucketCols() {
-    return this.getTTable().getSd().getBucketCols();
+    return getTTable().getSd().getBucketCols();
   }
 
   public List<Order> getSortCols() {
-    return this.getTTable().getSd().getSortCols();
+    return getTTable().getSd().getSortCols();
   }
 
   private static void getPartPaths(FileSystem fs, Path p, Vector<String> partPaths) throws IOException {
@@ -572,14 +530,14 @@ public class Table {
   }
 
   static final Pattern pat = Pattern.compile("([^/]+)=([^/]+)");
-  protected List<Partition> getPartitionsFromHDFS() throws HiveException {
+  public List<Partition> getPartitionsFromHDFS() throws HiveException {
     ArrayList<Partition> ret = new ArrayList<Partition> ();
     FileSystem fs = null;
     Vector<String> partPaths = new Vector<String>();
 
     try {
-      fs = FileSystem.get(this.getDataLocation(), Hive.get().getConf());
-      getPartPaths(fs, new Path(this.getDataLocation().getPath()), partPaths);
+      fs = FileSystem.get(getDataLocation(), Hive.get().getConf());
+      getPartPaths(fs, new Path(getDataLocation().getPath()), partPaths);
       for(String partPath: partPaths) {
         Path tmpPath = new Path(partPath);
         if(!fs.getFileStatus(tmpPath).isDir()) {
@@ -589,11 +547,10 @@ public class Table {
       }
     } catch (IOException e) {
       LOG.error(StringUtils.stringifyException(e));
-      throw new HiveException("DB Error: Table " + this.getDataLocation() + " message: " + e.getMessage());
+      throw new HiveException("DB Error: Table " + getDataLocation() + " message: " + e.getMessage());
     }
 
     return ret;
   }
   
-  
 };

+ 35 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.parse;
 import java.util.*;
 import java.io.File;
 import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -33,6 +34,7 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 
+
 public abstract class BaseSemanticAnalyzer {
   protected String scratchDir;
   protected int randomid;
@@ -79,7 +81,39 @@ public abstract class BaseSemanticAnalyzer {
     }
     return val;
   }
-  
+
+  public static String charSetString(String charSetName, String charSetString) 
+    throws SemanticException {
+    try
+    {
+      // The character set name starts with a _, so strip that
+      charSetName = charSetName.substring(1);
+      if (charSetString.charAt(0) == '\'')
+        return new String(unescapeSQLString(charSetString).getBytes(), charSetName);
+      else                                       // hex input is also supported
+      {
+        assert charSetString.charAt(0) == '0';
+        assert charSetString.charAt(1) == 'x';
+        charSetString = charSetString.substring(2);
+        
+        byte[] bArray = new byte[charSetString.length()/2];
+        int j = 0;
+        for (int i = 0; i < charSetString.length(); i += 2)
+        {
+          int val = Character.digit(charSetString.charAt(i), 16) * 16 + Character.digit(charSetString.charAt(i+1), 16);
+          if (val > 127)
+            val = val - 256;
+          bArray[j++] = new Integer(val).byteValue();
+        }
+
+        String res = new String(bArray, charSetName);
+        return res;
+      } 
+    } catch (UnsupportedEncodingException e) {
+      throw new SemanticException(e);
+    }
+  }
+
   @SuppressWarnings("nls")
   public static String unescapeSQLString(String b) {
     assert(b.charAt(0) == '\'');

+ 28 - 13
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java

@@ -18,9 +18,10 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.api.Constants;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Order;
 
 import org.antlr.runtime.tree.CommonTree;
 
@@ -35,11 +36,11 @@ import org.apache.hadoop.hive.ql.plan.showTablesDesc;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.serde.Constants;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import java.io.File;
 import java.util.*;
 
 public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
@@ -76,12 +77,12 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
        analyzeDropTable(ast);
     else if (ast.getToken().getType() == HiveParser.TOK_DESCTABLE)
     {
-      ctx.setResFile(new File(getTmpFileName()));
+      ctx.setResFile(new Path(getTmpFileName()));
       analyzeDescribeTable(ast);
     }
     else if (ast.getToken().getType() == HiveParser.TOK_SHOWTABLES)
     {
-      ctx.setResFile(new File(getTmpFileName()));
+      ctx.setResFile(new Path(getTmpFileName()));
       analyzeShowTables(ast);
     }
     else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_RENAME)
@@ -97,14 +98,14 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     List<FieldSchema> cols          = getColumns(colList);
     List<FieldSchema> partCols      = null;
     List<String>      bucketCols    = null;
-    List<String>      sortCols      = null;
+    List<Order>       sortCols      = null;
     int               numBuckets    = -1;
     String            fieldDelim    = null;
     String            collItemDelim = null;
     String            mapKeyDelim   = null;
     String            lineDelim     = null;
     String            comment       = null;
-    boolean           isCompressed  = false;
+    boolean           isSequenceFile  = false;
     String            location      = null;
 
     LOG.info("Creating table" + tableName);
@@ -125,7 +126,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
             numBuckets = (Integer.valueOf(child.getChild(1).getText())).intValue();
           else
           {
-            sortCols = getColumnNames((CommonTree)child.getChild(1));
+            sortCols = getColumnNamesOrder((CommonTree)child.getChild(1));
             numBuckets = (Integer.valueOf(child.getChild(2).getText())).intValue();
           }
           break;
@@ -151,8 +152,8 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
             }
           }
           break;
-        case HiveParser.TOK_TBLCOMPRESSED:
-          isCompressed = true;
+        case HiveParser.TOK_TBLSEQUENCEFILE:
+          isSequenceFile = true;
           break;
         case HiveParser.TOK_TABLELOCATION:
           location = unescapeSQLString(child.getChild(0).getText());
@@ -165,7 +166,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       new createTableDesc(tableName, isExt, cols, partCols, bucketCols, 
                           sortCols, numBuckets,
                           fieldDelim, collItemDelim, mapKeyDelim, lineDelim,
-                          comment, isCompressed, location);
+                          comment, isSequenceFile, location);
 
     validateCreateTable(crtTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(crtTblDesc), conf));
@@ -212,9 +213,9 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     if (crtTblDesc.getSortCols() != null)
     {
       // all columns in cluster and sort are valid columns
-      Iterator<String> sortCols = crtTblDesc.getSortCols().iterator();
+      Iterator<Order> sortCols = crtTblDesc.getSortCols().iterator();
       while (sortCols.hasNext()) {
-        String sortCol = sortCols.next();
+        String sortCol = sortCols.next().getCol();
         boolean found = false;
         Iterator<String> colNamesIter = colNames.iterator();
         while (colNamesIter.hasNext()) {
@@ -288,7 +289,21 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     int numCh = ast.getChildCount();
     for (int i = 0; i < numCh; i++) {
       CommonTree child = (CommonTree)ast.getChild(i);
-      colList.add(child.getChild(0).getText());
+      colList.add(child.getText());
+    }
+    return colList;
+  }
+
+  private List<Order> getColumnNamesOrder(CommonTree ast)
+  {
+    List<Order> colList = new ArrayList<Order>();
+    int numCh = ast.getChildCount();
+    for (int i = 0; i < numCh; i++) {
+      CommonTree child = (CommonTree)ast.getChild(i);
+      if (child.getToken().getType() == HiveParser.TOK_TABSORTCOLNAMEASC)
+        colList.add(new Order(child.getChild(0).getText(), 1));
+      else
+        colList.add(new Order(child.getChild(0).getText(), 0));
     }
     return colList;
   }

+ 2 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.parse;
 import java.io.File;
 
 import org.antlr.runtime.tree.CommonTree;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
@@ -44,7 +45,7 @@ public class ExplainSemanticAnalyzer extends BaseSemanticAnalyzer {
       extended = true;
     }
     
-    ctx.setResFile(new File(getTmpFileName()));
+    ctx.setResFile(new Path(getTmpFileName()));
     
     rootTasks.add(TaskFactory.get(new explainWork(ctx.getResFile(),
                                                   sem.getRootTasks(),

+ 46 - 10
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g

@@ -92,11 +92,14 @@ TOK_TABLEROWFORMATFIELD;
 TOK_TABLEROWFORMATCOLLITEMS;
 TOK_TABLEROWFORMATMAPKEYS;
 TOK_TABLEROWFORMATLINES;
-TOK_TBLCOMPRESSED;
+TOK_TBLSEQUENCEFILE;
 TOK_TABCOLNAME;
 TOK_TABLELOCATION;
 TOK_TABLESAMPLE;
 TOK_TMP_FILE;
+TOK_TABSORTCOLNAMEASC;
+TOK_TABSORTCOLNAMEDESC;
+TOK_CHARSETLITERAL;
 TOK_CREATEFUNCTION;
 TOK_EXPLAIN;
 }
@@ -146,9 +149,9 @@ ddlStatement
     ;
 
 createStatement
-    : KW_CREATE (ext=KW_EXTERNAL)? KW_TABLE name=Identifier LPAREN columnNameTypeList RPAREN tableComment? tablePartition? tableBuckets? tableRowFormat? tableCompress? tableLocation?
-    -> {$ext == null}? ^(TOK_CREATETABLE $name columnNameTypeList tableComment? tablePartition? tableBuckets? tableRowFormat? tableCompress? tableLocation?)
-    ->                 ^(TOK_CREATEEXTTABLE $name columnNameTypeList tableComment? tablePartition? tableBuckets? tableRowFormat? tableCompress? tableLocation?)
+    : KW_CREATE (ext=KW_EXTERNAL)? KW_TABLE name=Identifier LPAREN columnNameTypeList RPAREN tableComment? tablePartition? tableBuckets? tableRowFormat? tableFileFormat? tableLocation?
+    -> {$ext == null}? ^(TOK_CREATETABLE $name columnNameTypeList tableComment? tablePartition? tableBuckets? tableRowFormat? tableFileFormat? tableLocation?)
+    ->                 ^(TOK_CREATEEXTTABLE $name columnNameTypeList tableComment? tablePartition? tableBuckets? tableRowFormat? tableFileFormat? tableLocation?)
     ;
 
 dropStatement
@@ -200,7 +203,7 @@ tablePartition
 
 tableBuckets
     :
-      KW_CLUSTERED KW_BY LPAREN bucketCols=columnNameList RPAREN (KW_SORTED KW_BY LPAREN sortCols=columnNameList RPAREN)? KW_INTO num=Number KW_BUCKETS 
+      KW_CLUSTERED KW_BY LPAREN bucketCols=columnNameList RPAREN (KW_SORTED KW_BY LPAREN sortCols=columnNameOrderList RPAREN)? KW_INTO num=Number KW_BUCKETS 
     -> ^(TOK_TABLEBUCKETS $bucketCols $sortCols? $num)
     ;
 
@@ -234,9 +237,9 @@ tableRowFormatLinesIdentifier
     -> ^(TOK_TABLEROWFORMATLINES $linesIdnt)
     ;
 
-tableCompress
+tableFileFormat
     :
-      KW_STORED KW_AS KW_COMPRESSED  -> TOK_TBLCOMPRESSED
+      KW_STORED KW_AS KW_SEQUENCEFILE  -> TOK_TBLSEQUENCEFILE
     ;
 
 tableLocation
@@ -257,6 +260,16 @@ columnName
       Identifier
     ;
 
+columnNameOrderList
+    : columnNameOrder (COMMA columnNameOrder)* -> ^(TOK_TABCOLNAME columnNameOrder+)
+    ;
+
+columnNameOrder
+    : Identifier (asc=KW_ASC | desc=KW_DESC)? 
+    -> {$desc == null}? ^(TOK_TABSORTCOLNAMEASC Identifier)
+    ->                  ^(TOK_TABSORTCOLNAMEDESC Identifier)
+    ;
+
 columnNameType
     : colName=Identifier colType (KW_COMMENT comment=StringLiteral)?    
     -> {$comment == null}? ^(TOK_TABCOL $colName colType)
@@ -538,9 +551,14 @@ constant
     :
     Number
     | StringLiteral
+    | charSetStringLiteral
     | booleanValue 
-     ;
+    ;
 
+charSetStringLiteral
+    :
+    csName=CharSetName csLiteral=CharSetLiteral -> ^(TOK_CHARSETLITERAL $csName $csLiteral)
+    ;
 
 expression:
     precedenceOrExpression
@@ -772,7 +790,7 @@ KW_ITEMS: 'ITEMS';
 KW_KEYS: 'KEYS';
 KW_LINES: 'LINES';
 KW_STORED: 'STORED';
-KW_COMPRESSED: 'COMPRESSED';
+KW_SEQUENCEFILE: 'SEQUENCEFILE';
 KW_LOCATION: 'LOCATION';
 KW_TABLESAMPLE: 'TABLESAMPLE';
 KW_BUCKET: 'BUCKET';
@@ -824,6 +842,11 @@ Letter
     : 'a'..'z' | 'A'..'Z'
     ;
 
+fragment
+HexDigit
+    : 'a'..'f' | 'A'..'F' 
+    ;
+
 fragment
 Digit
     :
@@ -841,6 +864,12 @@ StringLiteral
     '\'' (~'\'')* '\'' ( '\'' (~'\'')* '\'' )*
     ;
 
+CharSetLiteral
+    :    
+    StringLiteral 
+    | '0' 'X' (HexDigit|Digit)+
+    ;
+
 Number
     :
     (Digit)+ ( DOT (Digit)* (Exponent)? | Exponent)?
@@ -848,7 +877,12 @@ Number
 
 Identifier
     :
-    (Letter | Digit | '_')+
+    (Letter | Digit) (Letter | Digit | '_')*
+    ;
+
+CharSetName
+    :
+    '_' (Letter | Digit | '_' | '-' | '.' | ':' )+
     ;
 
 WS  :  (' '|'\r'|'\t'|'\n') {$channel=HIDDEN;}
@@ -858,3 +892,5 @@ COMMENT
   : '--' (~('\n'|'\r'))*
     { $channel=HIDDEN; }
   ;
+
+

+ 3 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/InputSignature.java

@@ -23,6 +23,8 @@ import java.lang.Class;
 import java.lang.Object;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.ql.typeinfo.TypeInfoFactory;
 
 /**
  * The input signature of a function or operator. The signature basically consists
@@ -57,7 +59,7 @@ public class InputSignature {
     
     if (classList.length != 0) {
       for(Class<?> cl: classList) {
-        typeArray.add(TypeInfo.getPrimitiveTypeInfo(cl));
+        typeArray.add(TypeInfoFactory.getPrimitiveTypeInfo(cl));
       }
     }
   }

+ 39 - 18
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/PartitionPruner.java

@@ -27,13 +27,11 @@ package org.apache.hadoop.hive.ql.parse;
  */
 
 import java.util.*;
+
 import org.antlr.runtime.tree.*;
 
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
-import org.apache.hadoop.hive.ql.exec.HiveObject;
-import org.apache.hadoop.hive.ql.exec.LabeledCompositeHiveObject;
-import org.apache.hadoop.hive.ql.exec.PrimitiveHiveObject;
 import org.apache.hadoop.hive.ql.metadata.*;
 import org.apache.hadoop.hive.ql.plan.exprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeConstantDesc;
@@ -42,9 +40,16 @@ import org.apache.hadoop.hive.ql.plan.exprNodeFieldDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeFuncDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeIndexDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeNullDesc;
+import org.apache.hadoop.hive.ql.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.ql.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.hive.ql.udf.UDFOPAnd;
 import org.apache.hadoop.hive.ql.udf.UDFOPNot;
 import org.apache.hadoop.hive.ql.udf.UDFOPOr;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -111,8 +116,13 @@ public class PartitionPruner {
           desc = new exprNodeColumnDesc(String.class, colName); 
         } else {
           // might be a column from another table
-          TypeInfo typeInfo = new TypeInfo(this.metaData.getTableForAlias(tabAlias).getSerDe(), null);
-          desc = new exprNodeConstantDesc(typeInfo.getFieldType(colName), null);
+          try {
+            TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(
+                this.metaData.getTableForAlias(tabAlias).getDeserializer().getObjectInspector());
+            desc = new exprNodeConstantDesc(typeInfo.getStructFieldTypeInfo(colName), null);
+          } catch (SerDeException e){
+            throw new RuntimeException(e);
+          }
         }
         break;
       }
@@ -124,7 +134,9 @@ public class PartitionPruner {
         int childrenBegin = (isFunction ? 1 : 0);
         ArrayList<exprNodeDesc> children = new ArrayList<exprNodeDesc>(expr.getChildCount() - childrenBegin);
         for (int ci=childrenBegin; ci<expr.getChildCount(); ci++) {
-          children.add(genExprNodeDesc((CommonTree)expr.getChild(ci)));
+          exprNodeDesc child = genExprNodeDesc((CommonTree)expr.getChild(ci));
+          assert(child.getTypeInfo() != null);
+          children.add(child);
         }
 
         // Create function desc
@@ -198,27 +210,36 @@ public class PartitionPruner {
 
     HashSet<Partition> ret_parts = new HashSet<Partition>();
     try {
+      StructObjectInspector rowObjectInspector = (StructObjectInspector)this.tab.getDeserializer().getObjectInspector();
+      Object[] rowWithPart = new Object[2];
+      InspectableObject inspectableObject = new InspectableObject();
+      
       ExprNodeEvaluator evaluator = ExprNodeEvaluatorFactory.get(this.prunerExpr);
       for(Partition part: Hive.get().getPartitions(this.tab)) {
         // Set all the variables here
         LinkedHashMap<String, String> partSpec = part.getSpec();
 
         // Create the row object
-        String[] partNames = new String[partSpec.size()];
-        int i=0;
-        for(String name: partSpec.keySet()) {
-          partNames[i++] = name;
-        }
-        LabeledCompositeHiveObject hiveObject;
-        hiveObject = new LabeledCompositeHiveObject(partNames);
-        for(String s: partNames) {
-          hiveObject.addHiveObject(new PrimitiveHiveObject(partSpec.get(s)));
+        ArrayList<String> partNames = new ArrayList<String>();
+        ArrayList<String> partValues = new ArrayList<String>();
+        ArrayList<ObjectInspector> partObjectInspectors = new ArrayList<ObjectInspector>();
+        for(Map.Entry<String,String>entry : partSpec.entrySet()) {
+          partNames.add(entry.getKey());
+          partValues.add(entry.getValue());
+          partObjectInspectors.add(ObjectInspectorFactory.getStandardPrimitiveObjectInspector(String.class)); 
         }
+        StructObjectInspector partObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(partNames, partObjectInspectors);
+        
+        rowWithPart[1] = partValues;
+        ArrayList<StructObjectInspector> ois = new ArrayList<StructObjectInspector>(2);
+        ois.add(rowObjectInspector);
+        ois.add(partObjectInspector);
+        StructObjectInspector rowWithPartObjectInspector = ObjectInspectorFactory.getUnionStructObjectInspector(ois);
         
         // evaluate the expression tree
-        HiveObject r = evaluator.evaluate(hiveObject);
-        LOG.trace("prune result for partition " + partSpec + ": " + r.getJavaObject());
-        if (!Boolean.FALSE.equals(r.getJavaObject())) {
+        evaluator.evaluate(rowWithPart, rowWithPartObjectInspector, inspectableObject);
+        LOG.trace("prune result for partition " + partSpec + ": " + inspectableObject.o);
+        if (!Boolean.FALSE.equals(inspectableObject.o)) {
           LOG.debug("retained partition: " + partSpec);
           ret_parts.add(part);
         } else {

+ 4 - 24
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java

@@ -23,7 +23,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
-import org.apache.hadoop.hive.serde.ExpressionUtils;
 
 /**
  * Implementation of the Row Resolver
@@ -51,6 +50,7 @@ public class RowResolver {
   }
   
   public void put(String tab_alias, String col_alias, ColumnInfo colInfo) {
+    col_alias = col_alias.toLowerCase();
     if (rowSchema.getSignature() == null) {
       rowSchema.setSignature(new Vector<ColumnInfo>());
     }
@@ -75,34 +75,14 @@ public class RowResolver {
   }
 
   public ColumnInfo get(String tab_alias, String col_alias) {
+    col_alias = col_alias.toLowerCase();
     HashMap<String, ColumnInfo> f_map = rslvMap.get(tab_alias);
     if (f_map == null) {
       return null;
     }
-    ColumnInfo resInfo = f_map.get(col_alias.toLowerCase());
-    if(resInfo == null) {
-      // case insensitive search on column names but ANTLR Tokens are upppercase
-      // TODO: need to fix this in a better way
-      resInfo = f_map.get(col_alias);
-    }
-    if (resInfo == null) {
-      List<String> exprs = ExpressionUtils.decomposeComplexExpression(col_alias);
-      // Is this a complex field?
-      if (exprs.size() == 2) {
-        String topLevelField = exprs.get(0);
-        String suffix = exprs.get(1);
-        ColumnInfo info = f_map.get(topLevelField.toLowerCase());
-        if(info == null) {
-          info = f_map.get(topLevelField);
-        }
-        resInfo = new ColumnInfo(info.getInternalName() + suffix, 
-                                 info.getType().getFieldType(suffix),
-                                 info.getIsVirtual());
-      }
-    }
-    return resInfo; 
+    return f_map.get(col_alias);
   }
-   
+
   public Vector<ColumnInfo> getColumnInfos() {
     return rowSchema.getSignature();
   }

이 변경점에서 너무 많은 파일들이 변경되어 몇몇 파일들은 표시되지 않았습니다.