You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2012/11/08 10:44:29 UTC

svn commit: r1406984 [18/29] - in /hive/trunk: contrib/src/java/org/apache/hadoop/hive/contrib/genericudf/example/ contrib/src/java/org/apache/hadoop/hive/contrib/serde2/ contrib/src/java/org/apache/hadoop/hive/contrib/util/typedbytes/ contrib/src/test...

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java Thu Nov  8 09:44:19 2012
@@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.Constants;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -243,9 +243,9 @@ public class HiveAlterHandler implements
     String newPartLoc = null;
     // Set DDL time to now if not specified
     if (new_part.getParameters() == null ||
-        new_part.getParameters().get(Constants.DDL_TIME) == null ||
-        Integer.parseInt(new_part.getParameters().get(Constants.DDL_TIME)) == 0) {
-      new_part.putToParameters(Constants.DDL_TIME, Long.toString(System
+        new_part.getParameters().get(hive_metastoreConstants.DDL_TIME) == null ||
+        Integer.parseInt(new_part.getParameters().get(hive_metastoreConstants.DDL_TIME)) == 0) {
+      new_part.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
           .currentTimeMillis() / 1000));
     }
     //alter partition
@@ -386,9 +386,9 @@ public class HiveAlterHandler implements
       for (Partition tmpPart: new_parts) {
         // Set DDL time to now if not specified
         if (tmpPart.getParameters() == null ||
-            tmpPart.getParameters().get(Constants.DDL_TIME) == null ||
-            Integer.parseInt(tmpPart.getParameters().get(Constants.DDL_TIME)) == 0) {
-          tmpPart.putToParameters(Constants.DDL_TIME, Long.toString(System
+            tmpPart.getParameters().get(hive_metastoreConstants.DDL_TIME) == null ||
+            Integer.parseInt(tmpPart.getParameters().get(hive_metastoreConstants.DDL_TIME)) == 0) {
+          tmpPart.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
               .currentTimeMillis() / 1000));
         }
         Partition oldTmpPart = msdb.getPartition(dbname, name, tmpPart.getValues());

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Thu Nov  8 09:44:19 2012
@@ -59,7 +59,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
-import org.apache.hadoop.hive.metastore.api.Constants;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -1047,8 +1047,8 @@ public class HiveMetaStore extends Thrif
         long time = System.currentTimeMillis() / 1000;
         tbl.setCreateTime((int) time);
         if (tbl.getParameters() == null ||
-            tbl.getParameters().get(Constants.DDL_TIME) == null) {
-          tbl.putToParameters(Constants.DDL_TIME, Long.toString(time));
+            tbl.getParameters().get(hive_metastoreConstants.DDL_TIME) == null) {
+          tbl.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
         }
         ms.createTable(tbl);
         success = ms.commitTransaction();
@@ -1509,7 +1509,7 @@ public class HiveMetaStore extends Thrif
         // set create time
         long time = System.currentTimeMillis() / 1000;
         part.setCreateTime((int) time);
-        part.putToParameters(Constants.DDL_TIME, Long.toString(time));
+        part.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
 
         success = ms.addPartition(part);
         if (success) {
@@ -1719,8 +1719,8 @@ public class HiveMetaStore extends Thrif
         long time = System.currentTimeMillis() / 1000;
         part.setCreateTime((int) time);
         if (part.getParameters() == null ||
-            part.getParameters().get(Constants.DDL_TIME) == null) {
-          part.putToParameters(Constants.DDL_TIME, Long.toString(time));
+            part.getParameters().get(hive_metastoreConstants.DDL_TIME) == null) {
+          part.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
         }
 
         // Inherit table properties into partition properties.
@@ -2225,7 +2225,7 @@ public class HiveMetaStore extends Thrif
         throws InvalidOperationException, MetaException {
       startFunction("alter_index", ": db=" + dbname + " base_tbl=" + base_table_name
           + " idx=" + index_name + " newidx=" + newIndex.getIndexName());
-      newIndex.putToParameters(Constants.DDL_TIME, Long.toString(System
+      newIndex.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
           .currentTimeMillis() / 1000));
 
       boolean success = false;
@@ -2282,8 +2282,8 @@ public class HiveMetaStore extends Thrif
 
       // Update the time if it hasn't been specified.
       if (newTable.getParameters() == null ||
-          newTable.getParameters().get(Constants.DDL_TIME) == null) {
-        newTable.putToParameters(Constants.DDL_TIME, Long.toString(System
+          newTable.getParameters().get(hive_metastoreConstants.DDL_TIME) == null) {
+        newTable.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
             .currentTimeMillis() / 1000));
       }
       boolean success = false;
@@ -2863,7 +2863,7 @@ public class HiveMetaStore extends Thrif
         }
 
         index.setCreateTime((int) time);
-        index.putToParameters(Constants.DDL_TIME, Long.toString(time));
+        index.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
 
         ms.addIndex(index);
         success = ms.commitTransaction();

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java Thu Nov  8 09:44:19 2012
@@ -39,7 +39,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Constants;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
@@ -97,13 +97,13 @@ public class MetaStoreUtils {
     serdeInfo.setSerializationLib(LazySimpleSerDe.class.getName());
     serdeInfo.setParameters(new HashMap<String, String>());
     serdeInfo.getParameters().put(
-        org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
+        org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
 
     List<FieldSchema> fields = new ArrayList<FieldSchema>();
     sd.setCols(fields);
     for (String col : columns) {
       FieldSchema field = new FieldSchema(col,
-          org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "'default'");
+          org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME, "'default'");
       fields.add(field);
     }
 
@@ -111,7 +111,7 @@ public class MetaStoreUtils {
     for (String partCol : partCols) {
       FieldSchema part = new FieldSchema();
       part.setName(partCol);
-      part.setType(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME); // default
+      part.setType(org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME); // default
                                                                              // partition
                                                                              // key
       tTable.getPartitionKeys().add(part);
@@ -163,7 +163,7 @@ public class MetaStoreUtils {
   static public Deserializer getDeserializer(Configuration conf,
       Properties schema) throws MetaException {
     String lib = schema
-        .getProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB);
+        .getProperty(org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB);
     try {
       Deserializer deserializer = SerDeUtils.lookupDeserializer(lib);
       (deserializer).initialize(conf, schema);
@@ -370,37 +370,37 @@ public class MetaStoreUtils {
   static {
     typeToThriftTypeMap = new HashMap<String, String>();
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.BOOLEAN_TYPE_NAME, "bool");
+        org.apache.hadoop.hive.serde.serdeConstants.BOOLEAN_TYPE_NAME, "bool");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.TINYINT_TYPE_NAME, "byte");
+        org.apache.hadoop.hive.serde.serdeConstants.TINYINT_TYPE_NAME, "byte");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.SMALLINT_TYPE_NAME, "i16");
+        org.apache.hadoop.hive.serde.serdeConstants.SMALLINT_TYPE_NAME, "i16");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.INT_TYPE_NAME, "i32");
+        org.apache.hadoop.hive.serde.serdeConstants.INT_TYPE_NAME, "i32");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.BIGINT_TYPE_NAME, "i64");
+        org.apache.hadoop.hive.serde.serdeConstants.BIGINT_TYPE_NAME, "i64");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.DOUBLE_TYPE_NAME, "double");
+        org.apache.hadoop.hive.serde.serdeConstants.DOUBLE_TYPE_NAME, "double");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.FLOAT_TYPE_NAME, "float");
+        org.apache.hadoop.hive.serde.serdeConstants.FLOAT_TYPE_NAME, "float");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.LIST_TYPE_NAME, "list");
+        org.apache.hadoop.hive.serde.serdeConstants.LIST_TYPE_NAME, "list");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.MAP_TYPE_NAME, "map");
+        org.apache.hadoop.hive.serde.serdeConstants.MAP_TYPE_NAME, "map");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "string");
+        org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME, "string");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.BINARY_TYPE_NAME, "binary");
+        org.apache.hadoop.hive.serde.serdeConstants.BINARY_TYPE_NAME, "binary");
     // These 3 types are not supported yet.
     // We should define a complex type date in thrift that contains a single int
     // member, and DynamicSerDe
     // should convert it to date type at runtime.
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.DATE_TYPE_NAME, "date");
+        org.apache.hadoop.hive.serde.serdeConstants.DATE_TYPE_NAME, "date");
     typeToThriftTypeMap.put(
-        org.apache.hadoop.hive.serde.Constants.DATETIME_TYPE_NAME, "datetime");
+        org.apache.hadoop.hive.serde.serdeConstants.DATETIME_TYPE_NAME, "datetime");
     typeToThriftTypeMap
-        .put(org.apache.hadoop.hive.serde.Constants.TIMESTAMP_TYPE_NAME,
+        .put(org.apache.hadoop.hive.serde.serdeConstants.TIMESTAMP_TYPE_NAME,
             "timestamp");
   }
 
@@ -535,42 +535,42 @@ public class MetaStoreUtils {
     String inputFormat = sd.getInputFormat();
     if (inputFormat == null || inputFormat.length() == 0) {
       String tblInput =
-        schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_INPUT_FORMAT);
+        schema.getProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT);
       if (tblInput == null) {
         inputFormat = org.apache.hadoop.mapred.SequenceFileInputFormat.class.getName();
       } else {
         inputFormat = tblInput;
       }
     }
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_INPUT_FORMAT,
+    schema.setProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT,
         inputFormat);
 
     // OutputFormat
     String outputFormat = sd.getOutputFormat();
     if (outputFormat == null || outputFormat.length() == 0) {
       String tblOutput =
-        schema.getProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT);
+        schema.getProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_OUTPUT_FORMAT);
       if (tblOutput == null) {
         outputFormat = org.apache.hadoop.mapred.SequenceFileOutputFormat.class.getName();
       } else {
         outputFormat = tblOutput;
       }
     }
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT,
+    schema.setProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_OUTPUT_FORMAT,
         outputFormat);
 
     // Location
     if (sd.getLocation() != null) {
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_LOCATION,
+      schema.setProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION,
           sd.getLocation());
     }
 
     // Bucket count
-    schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_COUNT,
+    schema.setProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.BUCKET_COUNT,
         Integer.toString(sd.getNumBuckets()));
 
     if (sd.getBucketCols() != null && sd.getBucketCols().size() > 0) {
-      schema.setProperty(org.apache.hadoop.hive.metastore.api.Constants.BUCKET_FIELD_NAME,
+      schema.setProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.BUCKET_FIELD_NAME,
           sd.getBucketCols().get(0));
     }
 
@@ -580,9 +580,9 @@ public class MetaStoreUtils {
       // We should not update the following 3 values if SerDeInfo contains these.
       // This is to keep backward compatible with getSchema(), where these 3 keys
       // are updated after SerDeInfo properties got copied.
-      String cols = org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS;
-      String colTypes = org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMN_TYPES;
-      String parts = org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS;
+      String cols = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_COLUMNS;
+      String colTypes = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_COLUMN_TYPES;
+      String parts = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS;
 
       for (Map.Entry<String,String> param : sd.getSerdeInfo().getParameters().entrySet()) {
         String key = param.getKey();
@@ -594,7 +594,7 @@ public class MetaStoreUtils {
       }
 
       if (sd.getSerdeInfo().getSerializationLib() != null) {
-        schema.setProperty(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB,
+        schema.setProperty(org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB,
             sd.getSerdeInfo().getSerializationLib());
       }
     }
@@ -623,7 +623,7 @@ public class MetaStoreUtils {
         .getName();
     }
     schema.setProperty(
-      org.apache.hadoop.hive.metastore.api.Constants.FILE_INPUT_FORMAT,
+      org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT,
       inputFormat);
     String outputFormat = sd.getOutputFormat();
     if (outputFormat == null || outputFormat.length() == 0) {
@@ -631,24 +631,24 @@ public class MetaStoreUtils {
         .getName();
     }
     schema.setProperty(
-      org.apache.hadoop.hive.metastore.api.Constants.FILE_OUTPUT_FORMAT,
+      org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_OUTPUT_FORMAT,
       outputFormat);
 
     schema.setProperty(
-        org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME,
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_NAME,
         databaseName + "." + tableName);
 
     if (sd.getLocation() != null) {
       schema.setProperty(
-          org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_LOCATION,
+          org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION,
           sd.getLocation());
     }
     schema.setProperty(
-        org.apache.hadoop.hive.metastore.api.Constants.BUCKET_COUNT, Integer
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.BUCKET_COUNT, Integer
             .toString(sd.getNumBuckets()));
     if (sd.getBucketCols() != null && sd.getBucketCols().size() > 0) {
       schema.setProperty(
-          org.apache.hadoop.hive.metastore.api.Constants.BUCKET_FIELD_NAME, sd
+          org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.BUCKET_FIELD_NAME, sd
               .getBucketCols().get(0));
     }
     if (sd.getSerdeInfo() != null) {
@@ -658,7 +658,7 @@ public class MetaStoreUtils {
 
       if (sd.getSerdeInfo().getSerializationLib() != null) {
         schema.setProperty(
-            org.apache.hadoop.hive.serde.Constants.SERIALIZATION_LIB, sd
+            org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB, sd
                 .getSerdeInfo().getSerializationLib());
       }
     }
@@ -677,14 +677,14 @@ public class MetaStoreUtils {
     String colNames = colNameBuf.toString();
     String colTypes = colTypeBuf.toString();
     schema.setProperty(
-        org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMNS,
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_COLUMNS,
         colNames);
     schema.setProperty(
-        org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_COLUMN_TYPES,
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_COLUMN_TYPES,
         colTypes);
     if (sd.getCols() != null) {
       schema.setProperty(
-          org.apache.hadoop.hive.serde.Constants.SERIALIZATION_DDL,
+          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_DDL,
           getDDLFromFieldSchema(tableName, sd.getCols()));
     }
 
@@ -700,7 +700,7 @@ public class MetaStoreUtils {
     if (partString.length() > 0) {
       schema
           .setProperty(
-              org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS,
+              org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS,
               partString);
     }
 
@@ -909,7 +909,7 @@ public class MetaStoreUtils {
   public static boolean isArchived(
       org.apache.hadoop.hive.metastore.api.Partition part) {
     Map<String, String> params = part.getParameters();
-    if ("true".equalsIgnoreCase(params.get(Constants.IS_ARCHIVED))) {
+    if ("true".equalsIgnoreCase(params.get(hive_metastoreConstants.IS_ARCHIVED))) {
       return true;
     } else {
       return false;
@@ -920,7 +920,7 @@ public class MetaStoreUtils {
       org.apache.hadoop.hive.metastore.api.Partition part) {
     Map<String, String> params = part.getParameters();
     assert(isArchived(part));
-    String originalLocation = params.get(Constants.ORIGINAL_LOCATION);
+    String originalLocation = params.get(hive_metastoreConstants.ORIGINAL_LOCATION);
     assert( originalLocation != null);
 
     return new Path(originalLocation);
@@ -930,7 +930,7 @@ public class MetaStoreUtils {
     if (table == null) {
       return false;
     }
-    return (table.getParameters().get(Constants.META_TABLE_STORAGE) != null);
+    return (table.getParameters().get(hive_metastoreConstants.META_TABLE_STORAGE) != null);
   }
 
   /**

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java Thu Nov  8 09:44:19 2012
@@ -28,7 +28,7 @@ import org.antlr.runtime.ANTLRStringStre
 import org.antlr.runtime.CharStream;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.metastore.api.Constants;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
 
@@ -180,20 +180,20 @@ public class ExpressionTree {
 
     private String generateJDOFilterOverTables(Map<String, Object> params)
         throws MetaException {
-      if (keyName.equals(Constants.HIVE_FILTER_FIELD_OWNER)) {
+      if (keyName.equals(hive_metastoreConstants.HIVE_FILTER_FIELD_OWNER)) {
         keyName = "this.owner";
-      } else if (keyName.equals(Constants.HIVE_FILTER_FIELD_LAST_ACCESS)) {
+      } else if (keyName.equals(hive_metastoreConstants.HIVE_FILTER_FIELD_LAST_ACCESS)) {
         //lastAccessTime expects an integer, so we cannot use the "like operator"
         if (operator == Operator.LIKE) {
           throw new MetaException("Like is not supported for HIVE_FILTER_FIELD_LAST_ACCESS");
         }
         keyName = "this.lastAccessTime";
-      } else if (keyName.startsWith(Constants.HIVE_FILTER_FIELD_PARAMS)) {
+      } else if (keyName.startsWith(hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS)) {
         if (!TABLE_FILTER_OPS.contains(operator)) {
           throw new MetaException("Only " + TABLE_FILTER_OPS + " are supported " +
             "operators for HIVE_FILTER_FIELD_PARAMS");
         }
-        String paramKeyName = keyName.substring(Constants.HIVE_FILTER_FIELD_PARAMS.length());
+        String paramKeyName = keyName.substring(hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS.length());
         keyName = "this.parameters.get(\"" + paramKeyName + "\")";
         //value is persisted as a string in the db, so make sure it's a string here
         // in case we get an integer.
@@ -259,7 +259,7 @@ public class ExpressionTree {
 
       //Can only support partitions whose types are string
       if( ! table.getPartitionKeys().get(partitionColumnIndex).
-          getType().equals(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME) ) {
+          getType().equals(org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME) ) {
         throw new MetaException
         ("Filtering is supported only on partition keys of type string");
       }

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java Thu Nov  8 09:44:19 2012
@@ -59,7 +59,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.thrift.TException;
 
@@ -155,9 +155,9 @@ public abstract class TestHiveMetaStore 
       typ1.setName(typeName);
       typ1.setFields(new ArrayList<FieldSchema>(2));
       typ1.getFields().add(
-          new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
       typ1.getFields().add(
-          new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+          new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
       client.createType(typ1);
 
       Table tbl = new Table();
@@ -176,7 +176,7 @@ public abstract class TestHiveMetaStore 
       sd.getSerdeInfo().setName(tbl.getTableName());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters()
-          .put(Constants.SERIALIZATION_FORMAT, "1");
+          .put(serdeConstants.SERIALIZATION_FORMAT, "1");
       sd.setSortCols(new ArrayList<Order>());
 
       //skewed information
@@ -191,9 +191,9 @@ public abstract class TestHiveMetaStore 
 
       tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
       tbl.getPartitionKeys().add(
-          new FieldSchema("ds", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("ds", serdeConstants.STRING_TYPE_NAME, ""));
       tbl.getPartitionKeys().add(
-          new FieldSchema("hr", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("hr", serdeConstants.STRING_TYPE_NAME, ""));
 
       client.createTable(tbl);
 
@@ -603,8 +603,8 @@ public abstract class TestHiveMetaStore 
       client.createDatabase(db);
 
       ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+      cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
+      cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
 
       Table tbl = new Table();
       tbl.setDbName(dbName);
@@ -622,14 +622,14 @@ public abstract class TestHiveMetaStore 
       sd.getSerdeInfo().setName(tbl.getTableName());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters()
-          .put(Constants.SERIALIZATION_FORMAT, "1");
+          .put(serdeConstants.SERIALIZATION_FORMAT, "1");
       sd.setSortCols(new ArrayList<Order>());
 
       tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
       tbl.getPartitionKeys().add(
-          new FieldSchema("ds", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("ds", serdeConstants.STRING_TYPE_NAME, ""));
       tbl.getPartitionKeys().add(
-          new FieldSchema("hr", Constants.INT_TYPE_NAME, ""));
+          new FieldSchema("hr", serdeConstants.INT_TYPE_NAME, ""));
 
       client.createTable(tbl);
 
@@ -699,8 +699,8 @@ public abstract class TestHiveMetaStore 
       client.createDatabase(db);
 
       ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+      cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
+      cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
 
       Table tbl = new Table();
       tbl.setDbName(dbName);
@@ -718,14 +718,14 @@ public abstract class TestHiveMetaStore 
       sd.getSerdeInfo().setName(tbl.getTableName());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters()
-          .put(Constants.SERIALIZATION_FORMAT, "1");
+          .put(serdeConstants.SERIALIZATION_FORMAT, "1");
       sd.setSortCols(new ArrayList<Order>());
 
       tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
       tbl.getPartitionKeys().add(
-          new FieldSchema("ds", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("ds", serdeConstants.STRING_TYPE_NAME, ""));
       tbl.getPartitionKeys().add(
-          new FieldSchema("hr", Constants.INT_TYPE_NAME, ""));
+          new FieldSchema("hr", serdeConstants.INT_TYPE_NAME, ""));
 
       client.createTable(tbl);
 
@@ -943,23 +943,23 @@ public abstract class TestHiveMetaStore 
 
   public void testSimpleTypeApi() throws Exception {
     try {
-      client.dropType(Constants.INT_TYPE_NAME);
+      client.dropType(serdeConstants.INT_TYPE_NAME);
 
       Type typ1 = new Type();
-      typ1.setName(Constants.INT_TYPE_NAME);
+      typ1.setName(serdeConstants.INT_TYPE_NAME);
       boolean ret = client.createType(typ1);
       assertTrue("Unable to create type", ret);
 
-      Type typ1_2 = client.getType(Constants.INT_TYPE_NAME);
+      Type typ1_2 = client.getType(serdeConstants.INT_TYPE_NAME);
       assertNotNull(typ1_2);
       assertEquals(typ1.getName(), typ1_2.getName());
 
-      ret = client.dropType(Constants.INT_TYPE_NAME);
+      ret = client.dropType(serdeConstants.INT_TYPE_NAME);
       assertTrue("unable to drop type integer", ret);
 
       boolean exceptionThrown = false;
       try {
-        client.getType(Constants.INT_TYPE_NAME);
+        client.getType(serdeConstants.INT_TYPE_NAME);
       } catch (NoSuchObjectException e) {
         exceptionThrown = true;
       }
@@ -980,9 +980,9 @@ public abstract class TestHiveMetaStore 
       typ1.setName("Person");
       typ1.setFields(new ArrayList<FieldSchema>(2));
       typ1.getFields().add(
-          new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
       typ1.getFields().add(
-          new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+          new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
       boolean ret = client.createType(typ1);
       assertTrue("Unable to create type", ret);
 
@@ -999,7 +999,7 @@ public abstract class TestHiveMetaStore 
       fam.setName("Family");
       fam.setFields(new ArrayList<FieldSchema>(2));
       fam.getFields().add(
-          new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
       fam.getFields().add(
           new FieldSchema("members",
               MetaStoreUtils.getListType(typ1.getName()), ""));
@@ -1053,9 +1053,9 @@ public abstract class TestHiveMetaStore 
       typ1.setName(typeName);
       typ1.setFields(new ArrayList<FieldSchema>(2));
       typ1.getFields().add(
-          new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
       typ1.getFields().add(
-          new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+          new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
       client.createType(typ1);
 
       Table tbl = new Table();
@@ -1074,7 +1074,7 @@ public abstract class TestHiveMetaStore 
       sd.getSerdeInfo().setName(tbl.getTableName());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
+          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
       sd.getSerdeInfo().setSerializationLib(
           org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName());
       tbl.setPartitionKeys(new ArrayList<FieldSchema>());
@@ -1100,7 +1100,7 @@ public abstract class TestHiveMetaStore 
       assertNotNull(tbl2.getSd().getSerdeInfo());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
+          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
 
       tbl2.setTableName(tblName2);
       tbl2.setParameters(new HashMap<String, String>());
@@ -1429,8 +1429,8 @@ public abstract class TestHiveMetaStore 
       client.createDatabase(db);
 
       ArrayList<FieldSchema> invCols = new ArrayList<FieldSchema>(2);
-      invCols.add(new FieldSchema("n-ame", Constants.STRING_TYPE_NAME, ""));
-      invCols.add(new FieldSchema("in.come", Constants.INT_TYPE_NAME, ""));
+      invCols.add(new FieldSchema("n-ame", serdeConstants.STRING_TYPE_NAME, ""));
+      invCols.add(new FieldSchema("in.come", serdeConstants.INT_TYPE_NAME, ""));
 
       Table tbl = new Table();
       tbl.setDbName(dbName);
@@ -1448,7 +1448,7 @@ public abstract class TestHiveMetaStore 
       sd.getSerdeInfo().setName(tbl.getTableName());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
+          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
       boolean failed = false;
       try {
         client.createTable(tbl);
@@ -1460,8 +1460,8 @@ public abstract class TestHiveMetaStore 
             false);
       }
       ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+      cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
+      cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
 
       // create a valid table
       tbl.setTableName(tblName);
@@ -1565,9 +1565,9 @@ public abstract class TestHiveMetaStore 
       typ1.setName(typeName);
       typ1.setFields(new ArrayList<FieldSchema>(2));
       typ1.getFields().add(
-          new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
       typ1.getFields().add(
-          new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+          new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
       client.createType(typ1);
 
       Table tbl = new Table();
@@ -1586,17 +1586,17 @@ public abstract class TestHiveMetaStore 
       sd.getSerdeInfo().setName(tbl.getTableName());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "9");
+          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "9");
       sd.getSerdeInfo().setSerializationLib(
           org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName());
 
       tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
       tbl.getPartitionKeys().add(
           new FieldSchema("ds",
-              org.apache.hadoop.hive.serde.Constants.DATE_TYPE_NAME, ""));
+              org.apache.hadoop.hive.serde.serdeConstants.DATE_TYPE_NAME, ""));
       tbl.getPartitionKeys().add(
           new FieldSchema("hr",
-              org.apache.hadoop.hive.serde.Constants.INT_TYPE_NAME, ""));
+              org.apache.hadoop.hive.serde.serdeConstants.INT_TYPE_NAME, ""));
 
       client.createTable(tbl);
 
@@ -1613,9 +1613,9 @@ public abstract class TestHiveMetaStore 
 
       assertNotNull(tbl2.getPartitionKeys());
       assertEquals(2, tbl2.getPartitionKeys().size());
-      assertEquals(Constants.DATE_TYPE_NAME, tbl2.getPartitionKeys().get(0)
+      assertEquals(serdeConstants.DATE_TYPE_NAME, tbl2.getPartitionKeys().get(0)
           .getType());
-      assertEquals(Constants.INT_TYPE_NAME, tbl2.getPartitionKeys().get(1)
+      assertEquals(serdeConstants.INT_TYPE_NAME, tbl2.getPartitionKeys().get(1)
           .getType());
       assertEquals("ds", tbl2.getPartitionKeys().get(0).getName());
       assertEquals("hr", tbl2.getPartitionKeys().get(1).getName());
@@ -1670,8 +1670,8 @@ public abstract class TestHiveMetaStore 
       tbl.setTableName(tblName_1);
 
       ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+      cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
+      cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
 
       StorageDescriptor sd = new StorageDescriptor();
       sd.setSerdeInfo(new SerDeInfo());
@@ -1679,7 +1679,7 @@ public abstract class TestHiveMetaStore 
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "9");
+          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "9");
       sd.getSerdeInfo().setSerializationLib(
           org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName());
 
@@ -1715,12 +1715,12 @@ public abstract class TestHiveMetaStore 
         assertEquals(client.getConfigValue("hive.key4", val), "0");
         assertEquals(client.getConfigValue("hive.key5", val), val);
         assertEquals(client.getConfigValue(null, val), val);
-      } catch (TException e) {
-        e.printStackTrace();
-          assert (false);
       } catch (ConfigValSecurityException e) {
         e.printStackTrace();
         assert (false);
+      } catch (TException e) {
+        e.printStackTrace();
+        assert (false);
       }
     }
 
@@ -1728,12 +1728,12 @@ public abstract class TestHiveMetaStore 
     try {
       // Attempting to get the password should throw an exception
       client.getConfigValue("javax.jdo.option.ConnectionPassword", "password");
+    } catch (ConfigValSecurityException e) {
+      threwException = true;
     } catch (TException e) {
       e.printStackTrace();
       assert (false);
-    } catch (ConfigValSecurityException e) {
-      threwException = true;
-    }
+    } 
     assert (threwException);
   }
 
@@ -1742,7 +1742,7 @@ public abstract class TestHiveMetaStore 
   throws NoSuchObjectException, MetaException, TException {
     Partition part_get = client.getPartition(dbName, tblName, part.getValues());
     part.setCreateTime(part_get.getCreateTime());
-    part.putToParameters(org.apache.hadoop.hive.metastore.api.Constants.DDL_TIME, Long.toString(part_get.getCreateTime()));
+    part.putToParameters(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.DDL_TIME, Long.toString(part_get.getCreateTime()));
   }
 
   private static void silentDropDatabase(String dbName) throws MetaException, TException {
@@ -1797,13 +1797,13 @@ public abstract class TestHiveMetaStore 
     client.createDatabase(db);
 
     ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-    cols.add(new FieldSchema("c1", Constants.STRING_TYPE_NAME, ""));
-    cols.add(new FieldSchema("c2", Constants.INT_TYPE_NAME, ""));
+    cols.add(new FieldSchema("c1", serdeConstants.STRING_TYPE_NAME, ""));
+    cols.add(new FieldSchema("c2", serdeConstants.INT_TYPE_NAME, ""));
 
     ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>(3);
-    partCols.add(new FieldSchema("p1", Constants.STRING_TYPE_NAME, ""));
-    partCols.add(new FieldSchema("p2", Constants.STRING_TYPE_NAME, ""));
-    partCols.add(new FieldSchema("p3", Constants.INT_TYPE_NAME, ""));
+    partCols.add(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, ""));
+    partCols.add(new FieldSchema("p2", serdeConstants.STRING_TYPE_NAME, ""));
+    partCols.add(new FieldSchema("p3", serdeConstants.INT_TYPE_NAME, ""));
 
     Table tbl = new Table();
     tbl.setDbName(dbName);
@@ -1819,7 +1819,7 @@ public abstract class TestHiveMetaStore 
     sd.getSerdeInfo().setName(tbl.getTableName());
     sd.getSerdeInfo().setParameters(new HashMap<String, String>());
     sd.getSerdeInfo().getParameters()
-        .put(Constants.SERIALIZATION_FORMAT, "1");
+        .put(serdeConstants.SERIALIZATION_FORMAT, "1");
     sd.setSortCols(new ArrayList<Order>());
 
     tbl.setPartitionKeys(partCols);
@@ -1939,11 +1939,11 @@ public abstract class TestHiveMetaStore 
       client.createDatabase(db);
 
       ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("c1", Constants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("c2", Constants.INT_TYPE_NAME, ""));
+      cols.add(new FieldSchema("c1", serdeConstants.STRING_TYPE_NAME, ""));
+      cols.add(new FieldSchema("c2", serdeConstants.INT_TYPE_NAME, ""));
 
       ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>(1);
-      partCols.add(new FieldSchema("p1", Constants.STRING_TYPE_NAME, ""));
+      partCols.add(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, ""));
 
       Table tbl = new Table();
       tbl.setDbName(dbName);
@@ -1959,7 +1959,7 @@ public abstract class TestHiveMetaStore 
       sd.getSerdeInfo().setName(tbl.getTableName());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters()
-          .put(Constants.SERIALIZATION_FORMAT, "1");
+          .put(serdeConstants.SERIALIZATION_FORMAT, "1");
       sd.setSortCols(new ArrayList<Order>());
 
       tbl.setPartitionKeys(partCols);
@@ -2007,15 +2007,15 @@ public abstract class TestHiveMetaStore 
       createDb(dbName);
 
       ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("c1", Constants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("c2", Constants.INT_TYPE_NAME, ""));
+      cols.add(new FieldSchema("c1", serdeConstants.STRING_TYPE_NAME, ""));
+      cols.add(new FieldSchema("c2", serdeConstants.INT_TYPE_NAME, ""));
 
       ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>(2);
-      partCols.add(new FieldSchema("p1", Constants.STRING_TYPE_NAME, ""));
-      partCols.add(new FieldSchema("p2", Constants.STRING_TYPE_NAME, ""));
+      partCols.add(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, ""));
+      partCols.add(new FieldSchema("p2", serdeConstants.STRING_TYPE_NAME, ""));
 
       Map<String, String> serdParams = new HashMap<String, String>();
-      serdParams.put(Constants.SERIALIZATION_FORMAT, "1");
+      serdParams.put(serdeConstants.SERIALIZATION_FORMAT, "1");
       StorageDescriptor sd = createStorageDescriptor(tblName, partCols, null, serdParams);
 
       Table tbl = new Table();
@@ -2116,9 +2116,9 @@ public abstract class TestHiveMetaStore 
       String filter;
       //test owner
       //owner like ".*Owner.*" and owner like "test.*"
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_OWNER +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_OWNER +
           " like \".*Owner.*\" and " +
-          org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_OWNER +
+          org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_OWNER +
           " like  \"test.*\"";
       tableNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
       assertEquals(tableNames.size(), 3);
@@ -2127,7 +2127,7 @@ public abstract class TestHiveMetaStore 
       assert(tableNames.contains(table3.getTableName()));
 
       //owner = "testOwner1"
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_OWNER +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_OWNER +
           " = \"testOwner1\"";
 
       tableNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
@@ -2136,7 +2136,7 @@ public abstract class TestHiveMetaStore 
       assert(tableNames.contains(table3.getTableName()));
 
       //lastAccessTime < 90
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_LAST_ACCESS +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_LAST_ACCESS +
           " < 90";
 
       tableNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
@@ -2145,7 +2145,7 @@ public abstract class TestHiveMetaStore 
       assert(tableNames.contains(table3.getTableName()));
 
       //lastAccessTime > 90
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_LAST_ACCESS +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_LAST_ACCESS +
       " > 90";
 
       tableNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
@@ -2153,7 +2153,7 @@ public abstract class TestHiveMetaStore 
 
       //test params
       //test_param_2 = "50"
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_PARAMS +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS +
           "test_param_2 = \"50\"";
 
       tableNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
@@ -2162,38 +2162,38 @@ public abstract class TestHiveMetaStore 
       assert(tableNames.contains(table2.getTableName()));
 
       //test_param_2 = "75"
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_PARAMS +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS +
           "test_param_2 = \"75\"";
 
       tableNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
       assertEquals(0, tableNames.size());
 
       //key_dne = "50"
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_PARAMS +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS +
           "key_dne = \"50\"";
 
       tableNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
       assertEquals(0, tableNames.size());
 
       //test_param_1 != "yellow"
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_PARAMS +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS +
           "test_param_1 <> \"yellow\"";
 
       tableNames = client.listTableNamesByFilter(dbName, filter, (short) 2);
       assertEquals(2, tableNames.size());
 
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_PARAMS +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS +
           "test_param_1 != \"yellow\"";
 
       tableNames = client.listTableNamesByFilter(dbName, filter, (short) 2);
       assertEquals(2, tableNames.size());
 
       //owner = "testOwner1" and (lastAccessTime = 30 or test_param_1 = "hi")
-      filter = org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_OWNER +
+      filter = org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_OWNER +
         " = \"testOwner1\" and (" +
-        org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_LAST_ACCESS +
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_LAST_ACCESS +
         " = 30 or " +
-        org.apache.hadoop.hive.metastore.api.Constants.HIVE_FILTER_FIELD_PARAMS +
+        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS +
         "test_param_1 = \"hi\")";
       tableNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
 
@@ -2228,20 +2228,20 @@ public abstract class TestHiveMetaStore 
     int lastAccessTime, boolean hasSecondParam) throws Exception {
 
     ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-    cols.add(new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
-    cols.add(new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+    cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
+    cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
 
     Map<String, String> params = new HashMap<String, String>();
     params.put("sd_param_1", "Use this for comments etc");
 
     Map<String, String> serdParams = new HashMap<String, String>();
-    serdParams.put(Constants.SERIALIZATION_FORMAT, "1");
+    serdParams.put(serdeConstants.SERIALIZATION_FORMAT, "1");
 
     StorageDescriptor sd = createStorageDescriptor(tableName, cols, params, serdParams);
 
     Map<String, String> partitionKeys = new HashMap<String, String>();
-    partitionKeys.put("ds", Constants.STRING_TYPE_NAME);
-    partitionKeys.put("hr", Constants.INT_TYPE_NAME);
+    partitionKeys.put("ds", serdeConstants.STRING_TYPE_NAME);
+    partitionKeys.put("hr", serdeConstants.INT_TYPE_NAME);
 
     Map<String, String> tableParams =  new HashMap<String, String>();
     tableParams.put("test_param_1", "hi");
@@ -2278,14 +2278,14 @@ public abstract class TestHiveMetaStore 
       createDb(dbName);
 
       ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("c1", Constants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("c2", Constants.INT_TYPE_NAME, ""));
+      cols.add(new FieldSchema("c1", serdeConstants.STRING_TYPE_NAME, ""));
+      cols.add(new FieldSchema("c2", serdeConstants.INT_TYPE_NAME, ""));
 
       Map<String, String> params = new HashMap<String, String>();
       params.put("test_param_1", "Use this for comments etc");
 
       Map<String, String> serdParams = new HashMap<String, String>();
-      serdParams.put(Constants.SERIALIZATION_FORMAT, "1");
+      serdParams.put(serdeConstants.SERIALIZATION_FORMAT, "1");
 
       StorageDescriptor sd =  createStorageDescriptor(tblName, cols, params, serdParams);
 
@@ -2423,7 +2423,7 @@ public abstract class TestHiveMetaStore 
     sd.getSerdeInfo().setName(tableName);
     sd.getSerdeInfo().setParameters(serdParams);
     sd.getSerdeInfo().getParameters()
-        .put(Constants.SERIALIZATION_FORMAT, "1");
+        .put(serdeConstants.SERIALIZATION_FORMAT, "1");
     sd.setSortCols(new ArrayList<Order>());
 
     return sd;
@@ -2459,20 +2459,20 @@ public abstract class TestHiveMetaStore 
     createDb(dbName);
 
     Map<String, String> fields = new HashMap<String, String>();
-    fields.put("name", Constants.STRING_TYPE_NAME);
-    fields.put("income", Constants.INT_TYPE_NAME);
+    fields.put("name", serdeConstants.STRING_TYPE_NAME);
+    fields.put("income", serdeConstants.INT_TYPE_NAME);
 
     Type typ1 = createType(typeName, fields);
 
     Map<String , String> partitionKeys = new HashMap<String, String>();
-    partitionKeys.put("ds", Constants.STRING_TYPE_NAME);
-    partitionKeys.put("hr", Constants.STRING_TYPE_NAME);
+    partitionKeys.put("ds", serdeConstants.STRING_TYPE_NAME);
+    partitionKeys.put("hr", serdeConstants.STRING_TYPE_NAME);
 
     Map<String, String> params = new HashMap<String, String>();
     params.put("test_param_1", "Use this for comments etc");
 
     Map<String, String> serdParams = new HashMap<String, String>();
-    serdParams.put(Constants.SERIALIZATION_FORMAT, "1");
+    serdParams.put(serdeConstants.SERIALIZATION_FORMAT, "1");
 
     StorageDescriptor sd =  createStorageDescriptor(tblName, typ1.getFields(), params, serdParams);
 

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaTool.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaTool.java?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaTool.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaTool.java Thu Nov  8 09:44:19 2012
@@ -36,7 +36,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.tools.HiveMetaTool;
-import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils;
 import org.apache.hadoop.util.StringUtils;
 
@@ -92,9 +92,9 @@ public class TestHiveMetaTool extends Te
       typ1.setName(typeName);
       typ1.setFields(new ArrayList<FieldSchema>(2));
       typ1.getFields().add(
-          new FieldSchema("name", Constants.STRING_TYPE_NAME, ""));
+          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
       typ1.getFields().add(
-          new FieldSchema("income", Constants.INT_TYPE_NAME, ""));
+          new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
       client.createType(typ1);
 
       Table tbl = new Table();
@@ -113,7 +113,7 @@ public class TestHiveMetaTool extends Te
       sd.getSerdeInfo().setName(tbl.getTableName());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
+          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
       sd.getParameters().put(AvroSerdeUtils.SCHEMA_URL, avroUri);
       sd.getSerdeInfo().setSerializationLib(
           org.apache.hadoop.hive.serde2.avro.AvroSerDe.class.getName());
@@ -137,7 +137,7 @@ public class TestHiveMetaTool extends Te
       sd.getSerdeInfo().setName(tbl.getTableName());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, "1");
+          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
       sd.getParameters().put(AvroSerdeUtils.SCHEMA_URL, badAvroUri);
       sd.getSerdeInfo().setSerializationLib(
           org.apache.hadoop.hive.serde2.avro.AvroSerDe.class.getName());

Modified: hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_constants.cpp
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_constants.cpp?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_constants.cpp (original)
+++ hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_constants.cpp Thu Nov  8 09:44:19 2012
@@ -1,7 +1,8 @@
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * Autogenerated by Thrift Compiler (0.9.0)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
  */
 #include "queryplan_constants.h"
 

Modified: hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_constants.h
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_constants.h?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_constants.h (original)
+++ hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_constants.h Thu Nov  8 09:44:19 2012
@@ -1,7 +1,8 @@
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * Autogenerated by Thrift Compiler (0.9.0)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
  */
 #ifndef queryplan_CONSTANTS_H
 #define queryplan_CONSTANTS_H

Modified: hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_types.cpp
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_types.cpp?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_types.cpp (original)
+++ hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_types.cpp Thu Nov  8 09:44:19 2012
@@ -1,10 +1,13 @@
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * Autogenerated by Thrift Compiler (0.9.0)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
  */
 #include "queryplan_types.h"
 
+#include <algorithm>
+
 namespace Apache { namespace Hadoop { namespace Hive {
 
 int _kAdjacencyTypeValues[] = {
@@ -148,14 +151,14 @@ uint32_t Adjacency::read(::apache::thrif
             this->children.clear();
             uint32_t _size0;
             ::apache::thrift::protocol::TType _etype3;
-            iprot->readListBegin(_etype3, _size0);
+            xfer += iprot->readListBegin(_etype3, _size0);
             this->children.resize(_size0);
             uint32_t _i4;
             for (_i4 = 0; _i4 < _size0; ++_i4)
             {
               xfer += iprot->readString(this->children[_i4]);
             }
-            iprot->readListEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.children = true;
         } else {
@@ -187,9 +190,11 @@ uint32_t Adjacency::read(::apache::thrif
 uint32_t Adjacency::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   xfer += oprot->writeStructBegin("Adjacency");
+
   xfer += oprot->writeFieldBegin("node", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->node);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("children", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->children.size()));
@@ -201,14 +206,24 @@ uint32_t Adjacency::write(::apache::thri
     xfer += oprot->writeListEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("adjacencyType", ::apache::thrift::protocol::T_I32, 3);
   xfer += oprot->writeI32((int32_t)this->adjacencyType);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
+void swap(Adjacency &a, Adjacency &b) {
+  using ::std::swap;
+  swap(a.node, b.node);
+  swap(a.children, b.children);
+  swap(a.adjacencyType, b.adjacencyType);
+  swap(a.__isset, b.__isset);
+}
+
 const char* Graph::ascii_fingerprint = "1F7FB604B3EF8F7AFB5DEAD15F2FC0B5";
 const uint8_t Graph::binary_fingerprint[16] = {0x1F,0x7F,0xB6,0x04,0xB3,0xEF,0x8F,0x7A,0xFB,0x5D,0xEA,0xD1,0x5F,0x2F,0xC0,0xB5};
 
@@ -248,14 +263,14 @@ uint32_t Graph::read(::apache::thrift::p
             this->roots.clear();
             uint32_t _size8;
             ::apache::thrift::protocol::TType _etype11;
-            iprot->readListBegin(_etype11, _size8);
+            xfer += iprot->readListBegin(_etype11, _size8);
             this->roots.resize(_size8);
             uint32_t _i12;
             for (_i12 = 0; _i12 < _size8; ++_i12)
             {
               xfer += iprot->readString(this->roots[_i12]);
             }
-            iprot->readListEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.roots = true;
         } else {
@@ -268,14 +283,14 @@ uint32_t Graph::read(::apache::thrift::p
             this->adjacencyList.clear();
             uint32_t _size13;
             ::apache::thrift::protocol::TType _etype16;
-            iprot->readListBegin(_etype16, _size13);
+            xfer += iprot->readListBegin(_etype16, _size13);
             this->adjacencyList.resize(_size13);
             uint32_t _i17;
             for (_i17 = 0; _i17 < _size13; ++_i17)
             {
               xfer += this->adjacencyList[_i17].read(iprot);
             }
-            iprot->readListEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.adjacencyList = true;
         } else {
@@ -297,9 +312,11 @@ uint32_t Graph::read(::apache::thrift::p
 uint32_t Graph::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   xfer += oprot->writeStructBegin("Graph");
+
   xfer += oprot->writeFieldBegin("nodeType", ::apache::thrift::protocol::T_I32, 1);
   xfer += oprot->writeI32((int32_t)this->nodeType);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("roots", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->roots.size()));
@@ -311,6 +328,7 @@ uint32_t Graph::write(::apache::thrift::
     xfer += oprot->writeListEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("adjacencyList", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->adjacencyList.size()));
@@ -322,11 +340,20 @@ uint32_t Graph::write(::apache::thrift::
     xfer += oprot->writeListEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
+void swap(Graph &a, Graph &b) {
+  using ::std::swap;
+  swap(a.nodeType, b.nodeType);
+  swap(a.roots, b.roots);
+  swap(a.adjacencyList, b.adjacencyList);
+  swap(a.__isset, b.__isset);
+}
+
 const char* Operator::ascii_fingerprint = "30917C758A752485AF223B697479DE6C";
 const uint8_t Operator::binary_fingerprint[16] = {0x30,0x91,0x7C,0x75,0x8A,0x75,0x24,0x85,0xAF,0x22,0x3B,0x69,0x74,0x79,0xDE,0x6C};
 
@@ -375,7 +402,7 @@ uint32_t Operator::read(::apache::thrift
             uint32_t _size21;
             ::apache::thrift::protocol::TType _ktype22;
             ::apache::thrift::protocol::TType _vtype23;
-            iprot->readMapBegin(_ktype22, _vtype23, _size21);
+            xfer += iprot->readMapBegin(_ktype22, _vtype23, _size21);
             uint32_t _i25;
             for (_i25 = 0; _i25 < _size21; ++_i25)
             {
@@ -384,7 +411,7 @@ uint32_t Operator::read(::apache::thrift
               std::string& _val27 = this->operatorAttributes[_key26];
               xfer += iprot->readString(_val27);
             }
-            iprot->readMapEnd();
+            xfer += iprot->readMapEnd();
           }
           this->__isset.operatorAttributes = true;
         } else {
@@ -398,7 +425,7 @@ uint32_t Operator::read(::apache::thrift
             uint32_t _size28;
             ::apache::thrift::protocol::TType _ktype29;
             ::apache::thrift::protocol::TType _vtype30;
-            iprot->readMapBegin(_ktype29, _vtype30, _size28);
+            xfer += iprot->readMapBegin(_ktype29, _vtype30, _size28);
             uint32_t _i32;
             for (_i32 = 0; _i32 < _size28; ++_i32)
             {
@@ -407,7 +434,7 @@ uint32_t Operator::read(::apache::thrift
               int64_t& _val34 = this->operatorCounters[_key33];
               xfer += iprot->readI64(_val34);
             }
-            iprot->readMapEnd();
+            xfer += iprot->readMapEnd();
           }
           this->__isset.operatorCounters = true;
         } else {
@@ -445,12 +472,15 @@ uint32_t Operator::read(::apache::thrift
 uint32_t Operator::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   xfer += oprot->writeStructBegin("Operator");
+
   xfer += oprot->writeFieldBegin("operatorId", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->operatorId);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("operatorType", ::apache::thrift::protocol::T_I32, 2);
   xfer += oprot->writeI32((int32_t)this->operatorType);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("operatorAttributes", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->operatorAttributes.size()));
@@ -463,6 +493,7 @@ uint32_t Operator::write(::apache::thrif
     xfer += oprot->writeMapEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("operatorCounters", ::apache::thrift::protocol::T_MAP, 4);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->operatorCounters.size()));
@@ -475,17 +506,31 @@ uint32_t Operator::write(::apache::thrif
     xfer += oprot->writeMapEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("done", ::apache::thrift::protocol::T_BOOL, 5);
   xfer += oprot->writeBool(this->done);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("started", ::apache::thrift::protocol::T_BOOL, 6);
   xfer += oprot->writeBool(this->started);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
+void swap(Operator &a, Operator &b) {
+  using ::std::swap;
+  swap(a.operatorId, b.operatorId);
+  swap(a.operatorType, b.operatorType);
+  swap(a.operatorAttributes, b.operatorAttributes);
+  swap(a.operatorCounters, b.operatorCounters);
+  swap(a.done, b.done);
+  swap(a.started, b.started);
+  swap(a.__isset, b.__isset);
+}
+
 const char* Task::ascii_fingerprint = "AC741A136EFA51843AFC3A12F6A793D1";
 const uint8_t Task::binary_fingerprint[16] = {0xAC,0x74,0x1A,0x13,0x6E,0xFA,0x51,0x84,0x3A,0xFC,0x3A,0x12,0xF6,0xA7,0x93,0xD1};
 
@@ -534,7 +579,7 @@ uint32_t Task::read(::apache::thrift::pr
             uint32_t _size38;
             ::apache::thrift::protocol::TType _ktype39;
             ::apache::thrift::protocol::TType _vtype40;
-            iprot->readMapBegin(_ktype39, _vtype40, _size38);
+            xfer += iprot->readMapBegin(_ktype39, _vtype40, _size38);
             uint32_t _i42;
             for (_i42 = 0; _i42 < _size38; ++_i42)
             {
@@ -543,7 +588,7 @@ uint32_t Task::read(::apache::thrift::pr
               std::string& _val44 = this->taskAttributes[_key43];
               xfer += iprot->readString(_val44);
             }
-            iprot->readMapEnd();
+            xfer += iprot->readMapEnd();
           }
           this->__isset.taskAttributes = true;
         } else {
@@ -557,7 +602,7 @@ uint32_t Task::read(::apache::thrift::pr
             uint32_t _size45;
             ::apache::thrift::protocol::TType _ktype46;
             ::apache::thrift::protocol::TType _vtype47;
-            iprot->readMapBegin(_ktype46, _vtype47, _size45);
+            xfer += iprot->readMapBegin(_ktype46, _vtype47, _size45);
             uint32_t _i49;
             for (_i49 = 0; _i49 < _size45; ++_i49)
             {
@@ -566,7 +611,7 @@ uint32_t Task::read(::apache::thrift::pr
               int64_t& _val51 = this->taskCounters[_key50];
               xfer += iprot->readI64(_val51);
             }
-            iprot->readMapEnd();
+            xfer += iprot->readMapEnd();
           }
           this->__isset.taskCounters = true;
         } else {
@@ -587,14 +632,14 @@ uint32_t Task::read(::apache::thrift::pr
             this->operatorList.clear();
             uint32_t _size52;
             ::apache::thrift::protocol::TType _etype55;
-            iprot->readListBegin(_etype55, _size52);
+            xfer += iprot->readListBegin(_etype55, _size52);
             this->operatorList.resize(_size52);
             uint32_t _i56;
             for (_i56 = 0; _i56 < _size52; ++_i56)
             {
               xfer += this->operatorList[_i56].read(iprot);
             }
-            iprot->readListEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.operatorList = true;
         } else {
@@ -632,12 +677,15 @@ uint32_t Task::read(::apache::thrift::pr
 uint32_t Task::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   xfer += oprot->writeStructBegin("Task");
+
   xfer += oprot->writeFieldBegin("taskId", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->taskId);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("taskType", ::apache::thrift::protocol::T_I32, 2);
   xfer += oprot->writeI32((int32_t)this->taskType);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("taskAttributes", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->taskAttributes.size()));
@@ -650,6 +698,7 @@ uint32_t Task::write(::apache::thrift::p
     xfer += oprot->writeMapEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("taskCounters", ::apache::thrift::protocol::T_MAP, 4);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->taskCounters.size()));
@@ -662,6 +711,7 @@ uint32_t Task::write(::apache::thrift::p
     xfer += oprot->writeMapEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   if (this->__isset.operatorGraph) {
     xfer += oprot->writeFieldBegin("operatorGraph", ::apache::thrift::protocol::T_STRUCT, 5);
     xfer += this->operatorGraph.write(oprot);
@@ -683,14 +733,29 @@ uint32_t Task::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("done", ::apache::thrift::protocol::T_BOOL, 7);
   xfer += oprot->writeBool(this->done);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("started", ::apache::thrift::protocol::T_BOOL, 8);
   xfer += oprot->writeBool(this->started);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
+void swap(Task &a, Task &b) {
+  using ::std::swap;
+  swap(a.taskId, b.taskId);
+  swap(a.taskType, b.taskType);
+  swap(a.taskAttributes, b.taskAttributes);
+  swap(a.taskCounters, b.taskCounters);
+  swap(a.operatorGraph, b.operatorGraph);
+  swap(a.operatorList, b.operatorList);
+  swap(a.done, b.done);
+  swap(a.started, b.started);
+  swap(a.__isset, b.__isset);
+}
+
 const char* Stage::ascii_fingerprint = "86EA3C7B0690AFED21A3D479E2B32378";
 const uint8_t Stage::binary_fingerprint[16] = {0x86,0xEA,0x3C,0x7B,0x06,0x90,0xAF,0xED,0x21,0xA3,0xD4,0x79,0xE2,0xB3,0x23,0x78};
 
@@ -739,7 +804,7 @@ uint32_t Stage::read(::apache::thrift::p
             uint32_t _size61;
             ::apache::thrift::protocol::TType _ktype62;
             ::apache::thrift::protocol::TType _vtype63;
-            iprot->readMapBegin(_ktype62, _vtype63, _size61);
+            xfer += iprot->readMapBegin(_ktype62, _vtype63, _size61);
             uint32_t _i65;
             for (_i65 = 0; _i65 < _size61; ++_i65)
             {
@@ -748,7 +813,7 @@ uint32_t Stage::read(::apache::thrift::p
               std::string& _val67 = this->stageAttributes[_key66];
               xfer += iprot->readString(_val67);
             }
-            iprot->readMapEnd();
+            xfer += iprot->readMapEnd();
           }
           this->__isset.stageAttributes = true;
         } else {
@@ -762,7 +827,7 @@ uint32_t Stage::read(::apache::thrift::p
             uint32_t _size68;
             ::apache::thrift::protocol::TType _ktype69;
             ::apache::thrift::protocol::TType _vtype70;
-            iprot->readMapBegin(_ktype69, _vtype70, _size68);
+            xfer += iprot->readMapBegin(_ktype69, _vtype70, _size68);
             uint32_t _i72;
             for (_i72 = 0; _i72 < _size68; ++_i72)
             {
@@ -771,7 +836,7 @@ uint32_t Stage::read(::apache::thrift::p
               int64_t& _val74 = this->stageCounters[_key73];
               xfer += iprot->readI64(_val74);
             }
-            iprot->readMapEnd();
+            xfer += iprot->readMapEnd();
           }
           this->__isset.stageCounters = true;
         } else {
@@ -784,14 +849,14 @@ uint32_t Stage::read(::apache::thrift::p
             this->taskList.clear();
             uint32_t _size75;
             ::apache::thrift::protocol::TType _etype78;
-            iprot->readListBegin(_etype78, _size75);
+            xfer += iprot->readListBegin(_etype78, _size75);
             this->taskList.resize(_size75);
             uint32_t _i79;
             for (_i79 = 0; _i79 < _size75; ++_i79)
             {
               xfer += this->taskList[_i79].read(iprot);
             }
-            iprot->readListEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.taskList = true;
         } else {
@@ -829,12 +894,15 @@ uint32_t Stage::read(::apache::thrift::p
 uint32_t Stage::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   xfer += oprot->writeStructBegin("Stage");
+
   xfer += oprot->writeFieldBegin("stageId", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->stageId);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("stageType", ::apache::thrift::protocol::T_I32, 2);
   xfer += oprot->writeI32((int32_t)this->stageType);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("stageAttributes", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->stageAttributes.size()));
@@ -847,6 +915,7 @@ uint32_t Stage::write(::apache::thrift::
     xfer += oprot->writeMapEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("stageCounters", ::apache::thrift::protocol::T_MAP, 4);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->stageCounters.size()));
@@ -859,6 +928,7 @@ uint32_t Stage::write(::apache::thrift::
     xfer += oprot->writeMapEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("taskList", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->taskList.size()));
@@ -870,17 +940,32 @@ uint32_t Stage::write(::apache::thrift::
     xfer += oprot->writeListEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("done", ::apache::thrift::protocol::T_BOOL, 6);
   xfer += oprot->writeBool(this->done);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("started", ::apache::thrift::protocol::T_BOOL, 7);
   xfer += oprot->writeBool(this->started);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
+void swap(Stage &a, Stage &b) {
+  using ::std::swap;
+  swap(a.stageId, b.stageId);
+  swap(a.stageType, b.stageType);
+  swap(a.stageAttributes, b.stageAttributes);
+  swap(a.stageCounters, b.stageCounters);
+  swap(a.taskList, b.taskList);
+  swap(a.done, b.done);
+  swap(a.started, b.started);
+  swap(a.__isset, b.__isset);
+}
+
 const char* Query::ascii_fingerprint = "68300D63A5D40F2D17B9A9440FF626C1";
 const uint8_t Query::binary_fingerprint[16] = {0x68,0x30,0x0D,0x63,0xA5,0xD4,0x0F,0x2D,0x17,0xB9,0xA9,0x44,0x0F,0xF6,0x26,0xC1};
 
@@ -927,7 +1012,7 @@ uint32_t Query::read(::apache::thrift::p
             uint32_t _size83;
             ::apache::thrift::protocol::TType _ktype84;
             ::apache::thrift::protocol::TType _vtype85;
-            iprot->readMapBegin(_ktype84, _vtype85, _size83);
+            xfer += iprot->readMapBegin(_ktype84, _vtype85, _size83);
             uint32_t _i87;
             for (_i87 = 0; _i87 < _size83; ++_i87)
             {
@@ -936,7 +1021,7 @@ uint32_t Query::read(::apache::thrift::p
               std::string& _val89 = this->queryAttributes[_key88];
               xfer += iprot->readString(_val89);
             }
-            iprot->readMapEnd();
+            xfer += iprot->readMapEnd();
           }
           this->__isset.queryAttributes = true;
         } else {
@@ -950,7 +1035,7 @@ uint32_t Query::read(::apache::thrift::p
             uint32_t _size90;
             ::apache::thrift::protocol::TType _ktype91;
             ::apache::thrift::protocol::TType _vtype92;
-            iprot->readMapBegin(_ktype91, _vtype92, _size90);
+            xfer += iprot->readMapBegin(_ktype91, _vtype92, _size90);
             uint32_t _i94;
             for (_i94 = 0; _i94 < _size90; ++_i94)
             {
@@ -959,7 +1044,7 @@ uint32_t Query::read(::apache::thrift::p
               int64_t& _val96 = this->queryCounters[_key95];
               xfer += iprot->readI64(_val96);
             }
-            iprot->readMapEnd();
+            xfer += iprot->readMapEnd();
           }
           this->__isset.queryCounters = true;
         } else {
@@ -980,14 +1065,14 @@ uint32_t Query::read(::apache::thrift::p
             this->stageList.clear();
             uint32_t _size97;
             ::apache::thrift::protocol::TType _etype100;
-            iprot->readListBegin(_etype100, _size97);
+            xfer += iprot->readListBegin(_etype100, _size97);
             this->stageList.resize(_size97);
             uint32_t _i101;
             for (_i101 = 0; _i101 < _size97; ++_i101)
             {
               xfer += this->stageList[_i101].read(iprot);
             }
-            iprot->readListEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.stageList = true;
         } else {
@@ -1025,12 +1110,15 @@ uint32_t Query::read(::apache::thrift::p
 uint32_t Query::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   xfer += oprot->writeStructBegin("Query");
+
   xfer += oprot->writeFieldBegin("queryId", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->queryId);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("queryType", ::apache::thrift::protocol::T_STRING, 2);
   xfer += oprot->writeString(this->queryType);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("queryAttributes", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->queryAttributes.size()));
@@ -1043,6 +1131,7 @@ uint32_t Query::write(::apache::thrift::
     xfer += oprot->writeMapEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("queryCounters", ::apache::thrift::protocol::T_MAP, 4);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->queryCounters.size()));
@@ -1055,9 +1144,11 @@ uint32_t Query::write(::apache::thrift::
     xfer += oprot->writeMapEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("stageGraph", ::apache::thrift::protocol::T_STRUCT, 5);
   xfer += this->stageGraph.write(oprot);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("stageList", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->stageList.size()));
@@ -1069,17 +1160,33 @@ uint32_t Query::write(::apache::thrift::
     xfer += oprot->writeListEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("done", ::apache::thrift::protocol::T_BOOL, 7);
   xfer += oprot->writeBool(this->done);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("started", ::apache::thrift::protocol::T_BOOL, 8);
   xfer += oprot->writeBool(this->started);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
+void swap(Query &a, Query &b) {
+  using ::std::swap;
+  swap(a.queryId, b.queryId);
+  swap(a.queryType, b.queryType);
+  swap(a.queryAttributes, b.queryAttributes);
+  swap(a.queryCounters, b.queryCounters);
+  swap(a.stageGraph, b.stageGraph);
+  swap(a.stageList, b.stageList);
+  swap(a.done, b.done);
+  swap(a.started, b.started);
+  swap(a.__isset, b.__isset);
+}
+
 const char* QueryPlan::ascii_fingerprint = "3418D1B0C20C288C8406186700B772E3";
 const uint8_t QueryPlan::binary_fingerprint[16] = {0x34,0x18,0xD1,0xB0,0xC2,0x0C,0x28,0x8C,0x84,0x06,0x18,0x67,0x00,0xB7,0x72,0xE3};
 
@@ -1109,14 +1216,14 @@ uint32_t QueryPlan::read(::apache::thrif
             this->queries.clear();
             uint32_t _size105;
             ::apache::thrift::protocol::TType _etype108;
-            iprot->readListBegin(_etype108, _size105);
+            xfer += iprot->readListBegin(_etype108, _size105);
             this->queries.resize(_size105);
             uint32_t _i109;
             for (_i109 = 0; _i109 < _size105; ++_i109)
             {
               xfer += this->queries[_i109].read(iprot);
             }
-            iprot->readListEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.queries = true;
         } else {
@@ -1154,6 +1261,7 @@ uint32_t QueryPlan::read(::apache::thrif
 uint32_t QueryPlan::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   xfer += oprot->writeStructBegin("QueryPlan");
+
   xfer += oprot->writeFieldBegin("queries", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->queries.size()));
@@ -1165,15 +1273,26 @@ uint32_t QueryPlan::write(::apache::thri
     xfer += oprot->writeListEnd();
   }
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("done", ::apache::thrift::protocol::T_BOOL, 2);
   xfer += oprot->writeBool(this->done);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldBegin("started", ::apache::thrift::protocol::T_BOOL, 3);
   xfer += oprot->writeBool(this->started);
   xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
+void swap(QueryPlan &a, QueryPlan &b) {
+  using ::std::swap;
+  swap(a.queries, b.queries);
+  swap(a.done, b.done);
+  swap(a.started, b.started);
+  swap(a.__isset, b.__isset);
+}
+
 }}} // namespace

Modified: hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_types.h
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_types.h?rev=1406984&r1=1406983&r2=1406984&view=diff
==============================================================================
--- hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_types.h (original)
+++ hive/trunk/ql/src/gen/thrift/gen-cpp/queryplan_types.h Thu Nov  8 09:44:19 2012
@@ -1,15 +1,16 @@
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * Autogenerated by Thrift Compiler (0.9.0)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
  */
 #ifndef queryplan_TYPES_H
 #define queryplan_TYPES_H
 
-#include <Thrift.h>
-#include <TApplicationException.h>
-#include <protocol/TProtocol.h>
-#include <transport/TTransport.h>
+#include <thrift/Thrift.h>
+#include <thrift/TApplicationException.h>
+#include <thrift/protocol/TProtocol.h>
+#include <thrift/transport/TTransport.h>
 
 
 
@@ -100,7 +101,7 @@ class Adjacency {
   static const char* ascii_fingerprint; // = "BC4F8C394677A1003AA9F56ED26D8204";
   static const uint8_t binary_fingerprint[16]; // = {0xBC,0x4F,0x8C,0x39,0x46,0x77,0xA1,0x00,0x3A,0xA9,0xF5,0x6E,0xD2,0x6D,0x82,0x04};
 
-  Adjacency() : node("") {
+  Adjacency() : node(), adjacencyType((AdjacencyType::type)0) {
   }
 
   virtual ~Adjacency() throw() {}
@@ -144,6 +145,8 @@ class Adjacency {
 
 };
 
+void swap(Adjacency &a, Adjacency &b);
+
 typedef struct _Graph__isset {
   _Graph__isset() : nodeType(false), roots(false), adjacencyList(false) {}
   bool nodeType;
@@ -157,7 +160,7 @@ class Graph {
   static const char* ascii_fingerprint; // = "1F7FB604B3EF8F7AFB5DEAD15F2FC0B5";
   static const uint8_t binary_fingerprint[16]; // = {0x1F,0x7F,0xB6,0x04,0xB3,0xEF,0x8F,0x7A,0xFB,0x5D,0xEA,0xD1,0x5F,0x2F,0xC0,0xB5};
 
-  Graph() {
+  Graph() : nodeType((NodeType::type)0) {
   }
 
   virtual ~Graph() throw() {}
@@ -201,6 +204,8 @@ class Graph {
 
 };
 
+void swap(Graph &a, Graph &b);
+
 typedef struct _Operator__isset {
   _Operator__isset() : operatorId(false), operatorType(false), operatorAttributes(false), operatorCounters(false), done(false), started(false) {}
   bool operatorId;
@@ -217,7 +222,7 @@ class Operator {
   static const char* ascii_fingerprint; // = "30917C758A752485AF223B697479DE6C";
   static const uint8_t binary_fingerprint[16]; // = {0x30,0x91,0x7C,0x75,0x8A,0x75,0x24,0x85,0xAF,0x22,0x3B,0x69,0x74,0x79,0xDE,0x6C};
 
-  Operator() : operatorId(""), done(0), started(0) {
+  Operator() : operatorId(), operatorType((OperatorType::type)0), done(0), started(0) {
   }
 
   virtual ~Operator() throw() {}
@@ -282,6 +287,8 @@ class Operator {
 
 };
 
+void swap(Operator &a, Operator &b);
+
 typedef struct _Task__isset {
   _Task__isset() : taskId(false), taskType(false), taskAttributes(false), taskCounters(false), operatorGraph(false), operatorList(false), done(false), started(false) {}
   bool taskId;
@@ -300,7 +307,7 @@ class Task {
   static const char* ascii_fingerprint; // = "AC741A136EFA51843AFC3A12F6A793D1";
   static const uint8_t binary_fingerprint[16]; // = {0xAC,0x74,0x1A,0x13,0x6E,0xFA,0x51,0x84,0x3A,0xFC,0x3A,0x12,0xF6,0xA7,0x93,0xD1};
 
-  Task() : taskId(""), done(0), started(0) {
+  Task() : taskId(), taskType((TaskType::type)0), done(0), started(0) {
   }
 
   virtual ~Task() throw() {}
@@ -385,6 +392,8 @@ class Task {
 
 };
 
+void swap(Task &a, Task &b);
+
 typedef struct _Stage__isset {
   _Stage__isset() : stageId(false), stageType(false), stageAttributes(false), stageCounters(false), taskList(false), done(false), started(false) {}
   bool stageId;
@@ -402,7 +411,7 @@ class Stage {
   static const char* ascii_fingerprint; // = "86EA3C7B0690AFED21A3D479E2B32378";
   static const uint8_t binary_fingerprint[16]; // = {0x86,0xEA,0x3C,0x7B,0x06,0x90,0xAF,0xED,0x21,0xA3,0xD4,0x79,0xE2,0xB3,0x23,0x78};
 
-  Stage() : stageId(""), done(0), started(0) {
+  Stage() : stageId(), stageType((StageType::type)0), done(0), started(0) {
   }
 
   virtual ~Stage() throw() {}
@@ -474,6 +483,8 @@ class Stage {
 
 };
 
+void swap(Stage &a, Stage &b);
+
 typedef struct _Query__isset {
   _Query__isset() : queryId(false), queryType(false), queryAttributes(false), queryCounters(false), stageGraph(false), stageList(false), done(false), started(false) {}
   bool queryId;
@@ -492,7 +503,7 @@ class Query {
   static const char* ascii_fingerprint; // = "68300D63A5D40F2D17B9A9440FF626C1";
   static const uint8_t binary_fingerprint[16]; // = {0x68,0x30,0x0D,0x63,0xA5,0xD4,0x0F,0x2D,0x17,0xB9,0xA9,0x44,0x0F,0xF6,0x26,0xC1};
 
-  Query() : queryId(""), queryType(""), done(0), started(0) {
+  Query() : queryId(), queryType(), done(0), started(0) {
   }
 
   virtual ~Query() throw() {}
@@ -571,6 +582,8 @@ class Query {
 
 };
 
+void swap(Query &a, Query &b);
+
 typedef struct _QueryPlan__isset {
   _QueryPlan__isset() : queries(false), done(false), started(false) {}
   bool queries;
@@ -628,6 +641,8 @@ class QueryPlan {
 
 };
 
+void swap(QueryPlan &a, QueryPlan &b);
+
 }}} // namespace
 
 #endif