You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2010/01/21 11:38:15 UTC

svn commit: r901644 [22/37] - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/ ql/src/java/org/apache/hadoop/hive/ql/history/ ql/src/jav...

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java Thu Jan 21 10:37:58 2010
@@ -18,182 +18,203 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
-import java.util.*;
-import java.io.*;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Properties;
+import java.util.Vector;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.parse.TypeCheckProcFactory;
 import org.apache.hadoop.hive.serde.Constants;
-import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
-import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
+import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
 import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
+import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
+import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
+import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.hive.serde2.Deserializer;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 
 public class PlanUtils {
 
-  protected final static Log LOG = LogFactory.getLog("org.apache.hadoop.hive.ql.plan.PlanUtils");
+  protected final static Log LOG = LogFactory
+      .getLog("org.apache.hadoop.hive.ql.plan.PlanUtils");
 
-  public static enum ExpressionTypes {FIELD, JEXL};
+  public static enum ExpressionTypes {
+    FIELD, JEXL
+  };
 
   @SuppressWarnings("nls")
   public static mapredWork getMapRedWork() {
-    return new mapredWork("",
-                          new LinkedHashMap<String, ArrayList<String>> (),
-                          new LinkedHashMap<String, partitionDesc> (),
-                          new LinkedHashMap<String, Operator<? extends Serializable>> (),
-                          new tableDesc(),
-                          new ArrayList<tableDesc> (),
-                          null,
-                          Integer.valueOf (1), null);
+    return new mapredWork("", new LinkedHashMap<String, ArrayList<String>>(),
+        new LinkedHashMap<String, partitionDesc>(),
+        new LinkedHashMap<String, Operator<? extends Serializable>>(),
+        new tableDesc(), new ArrayList<tableDesc>(), null, Integer.valueOf(1),
+        null);
   }
 
   /**
-   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the separatorCode
-   * and column names (comma separated string).
+   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the
+   * separatorCode and column names (comma separated string).
    */
-  public static tableDesc getDefaultTableDesc(String separatorCode, String columns) {
+  public static tableDesc getDefaultTableDesc(String separatorCode,
+      String columns) {
     return getDefaultTableDesc(separatorCode, columns, false);
   }
 
   /**
-   * Generate the table descriptor of given serde with the separatorCode
-   * and column names (comma separated string).
+   * Generate the table descriptor of given serde with the separatorCode and
+   * column names (comma separated string).
    */
-  public static tableDesc getTableDesc(Class<? extends Deserializer> serdeClass, String separatorCode, String columns) {
+  public static tableDesc getTableDesc(
+      Class<? extends Deserializer> serdeClass, String separatorCode,
+      String columns) {
     return getTableDesc(serdeClass, separatorCode, columns, false);
   }
 
   /**
-   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the separatorCode
-   * and column names (comma separated string), and whether the last column should take
-   * the rest of the line.
-   */
-  public static tableDesc getDefaultTableDesc(String separatorCode, String columns,
-      boolean lastColumnTakesRestOfTheLine) {
-    return getDefaultTableDesc(separatorCode, columns, null, lastColumnTakesRestOfTheLine);
+   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the
+   * separatorCode and column names (comma separated string), and whether the
+   * last column should take the rest of the line.
+   */
+  public static tableDesc getDefaultTableDesc(String separatorCode,
+      String columns, boolean lastColumnTakesRestOfTheLine) {
+    return getDefaultTableDesc(separatorCode, columns, null,
+        lastColumnTakesRestOfTheLine);
   }
 
   /**
    * Generate the table descriptor of the serde specified with the separatorCode
-   * and column names (comma separated string), and whether the last column should take
-   * the rest of the line.
+   * and column names (comma separated string), and whether the last column
+   * should take the rest of the line.
    */
-  public static tableDesc getTableDesc(Class<? extends Deserializer> serdeClass,
-                                       String separatorCode, String columns,
-                                       boolean lastColumnTakesRestOfTheLine) {
-    return getTableDesc(serdeClass, separatorCode, columns, null, lastColumnTakesRestOfTheLine);
+  public static tableDesc getTableDesc(
+      Class<? extends Deserializer> serdeClass, String separatorCode,
+      String columns, boolean lastColumnTakesRestOfTheLine) {
+    return getTableDesc(serdeClass, separatorCode, columns, null,
+        lastColumnTakesRestOfTheLine);
   }
 
   /**
-   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the separatorCode
-   * and column names (comma separated string), and whether the last column should take
-   * the rest of the line.
+   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the
+   * separatorCode and column names (comma separated string), and whether the
+   * last column should take the rest of the line.
    */
-  public static tableDesc getDefaultTableDesc(String separatorCode, String columns, String columnTypes,
-      boolean lastColumnTakesRestOfTheLine) {
-    return getTableDesc(LazySimpleSerDe.class, separatorCode, columns, columnTypes,
-                        lastColumnTakesRestOfTheLine);
+  public static tableDesc getDefaultTableDesc(String separatorCode,
+      String columns, String columnTypes, boolean lastColumnTakesRestOfTheLine) {
+    return getTableDesc(LazySimpleSerDe.class, separatorCode, columns,
+        columnTypes, lastColumnTakesRestOfTheLine);
   }
 
-  public static tableDesc getTableDesc(Class<? extends Deserializer> serdeClass,
-                                       String separatorCode, String columns, String columnTypes,
-      boolean lastColumnTakesRestOfTheLine) {
-    return getTableDesc(serdeClass, separatorCode, columns, columnTypes, lastColumnTakesRestOfTheLine, false);
+  public static tableDesc getTableDesc(
+      Class<? extends Deserializer> serdeClass, String separatorCode,
+      String columns, String columnTypes, boolean lastColumnTakesRestOfTheLine) {
+    return getTableDesc(serdeClass, separatorCode, columns, columnTypes,
+        lastColumnTakesRestOfTheLine, false);
   }
 
-  public static tableDesc getTableDesc(Class<? extends Deserializer> serdeClass,
-                                       String separatorCode, String columns, String columnTypes,
-                                       boolean lastColumnTakesRestOfTheLine, boolean useJSONForLazy) {
+  public static tableDesc getTableDesc(
+      Class<? extends Deserializer> serdeClass, String separatorCode,
+      String columns, String columnTypes, boolean lastColumnTakesRestOfTheLine,
+      boolean useJSONForLazy) {
 
     Properties properties = Utilities.makeProperties(
-      Constants.SERIALIZATION_FORMAT, separatorCode,
-      Constants.LIST_COLUMNS, columns);
+        Constants.SERIALIZATION_FORMAT, separatorCode, Constants.LIST_COLUMNS,
+        columns);
 
-    if ( ! separatorCode.equals(Integer.toString(Utilities.ctrlaCode)) )
+    if (!separatorCode.equals(Integer.toString(Utilities.ctrlaCode))) {
       properties.setProperty(Constants.FIELD_DELIM, separatorCode);
+    }
 
-    if (columnTypes != null)
+    if (columnTypes != null) {
       properties.setProperty(Constants.LIST_COLUMN_TYPES, columnTypes);
+    }
 
     if (lastColumnTakesRestOfTheLine) {
-      properties.setProperty(
-          Constants.SERIALIZATION_LAST_COLUMN_TAKES_REST,
+      properties.setProperty(Constants.SERIALIZATION_LAST_COLUMN_TAKES_REST,
           "true");
     }
 
-    // It is not a very clean way, and should be modified later - due to compatiblity reasons,
-    // user sees the results as json for custom scripts and has no way for specifying that.
+    // It is not a very clean way, and should be modified later - due to
+    // compatiblity reasons,
+    // user sees the results as json for custom scripts and has no way for
+    // specifying that.
     // Right now, it is hard-coded in the code
-    if (useJSONForLazy)
-      properties.setProperty(
-          Constants.SERIALIZATION_USE_JSON_OBJECTS,
-          "true");
-
+    if (useJSONForLazy) {
+      properties.setProperty(Constants.SERIALIZATION_USE_JSON_OBJECTS, "true");
+    }
 
-    return new tableDesc(
-      serdeClass,
-      TextInputFormat.class,
-      IgnoreKeyTextOutputFormat.class,
-      properties);
+    return new tableDesc(serdeClass, TextInputFormat.class,
+        IgnoreKeyTextOutputFormat.class, properties);
   }
 
   /**
    * Generate a table descriptor from a createTableDesc.
    */
-  public static tableDesc getTableDesc(createTableDesc crtTblDesc, String cols, String colTypes) {
+  public static tableDesc getTableDesc(createTableDesc crtTblDesc, String cols,
+      String colTypes) {
 
     Class<? extends Deserializer> serdeClass = LazySimpleSerDe.class;
-    String separatorCode                     = Integer.toString(Utilities.ctrlaCode);
-    String columns                           = cols;
-    String columnTypes                       = colTypes;
-    boolean lastColumnTakesRestOfTheLine     = false;
+    String separatorCode = Integer.toString(Utilities.ctrlaCode);
+    String columns = cols;
+    String columnTypes = colTypes;
+    boolean lastColumnTakesRestOfTheLine = false;
     tableDesc ret;
 
     try {
-      if ( crtTblDesc.getSerName() != null ) {
+      if (crtTblDesc.getSerName() != null) {
         Class c = Class.forName(crtTblDesc.getSerName());
         serdeClass = c;
       }
 
-      if ( crtTblDesc.getFieldDelim() != null )
+      if (crtTblDesc.getFieldDelim() != null) {
         separatorCode = crtTblDesc.getFieldDelim();
+      }
 
       ret = getTableDesc(serdeClass, separatorCode, columns, columnTypes,
-                         lastColumnTakesRestOfTheLine, false);
+          lastColumnTakesRestOfTheLine, false);
 
       // set other table properties
       Properties properties = ret.getProperties();
 
-      if ( crtTblDesc.getCollItemDelim() != null )
-        properties.setProperty(Constants.COLLECTION_DELIM, crtTblDesc.getCollItemDelim());
+      if (crtTblDesc.getCollItemDelim() != null) {
+        properties.setProperty(Constants.COLLECTION_DELIM, crtTblDesc
+            .getCollItemDelim());
+      }
 
-      if ( crtTblDesc.getMapKeyDelim() != null )
-        properties.setProperty(Constants.MAPKEY_DELIM, crtTblDesc.getMapKeyDelim());
+      if (crtTblDesc.getMapKeyDelim() != null) {
+        properties.setProperty(Constants.MAPKEY_DELIM, crtTblDesc
+            .getMapKeyDelim());
+      }
 
-      if ( crtTblDesc.getFieldEscape() != null )
-        properties.setProperty(Constants.ESCAPE_CHAR, crtTblDesc.getFieldEscape());
+      if (crtTblDesc.getFieldEscape() != null) {
+        properties.setProperty(Constants.ESCAPE_CHAR, crtTblDesc
+            .getFieldEscape());
+      }
 
-      if ( crtTblDesc.getLineDelim() != null )
+      if (crtTblDesc.getLineDelim() != null) {
         properties.setProperty(Constants.LINE_DELIM, crtTblDesc.getLineDelim());
+      }
 
-      // replace the default input & output file format with those found in crtTblDesc
+      // replace the default input & output file format with those found in
+      // crtTblDesc
       Class c1 = Class.forName(crtTblDesc.getInputFormat());
       Class c2 = Class.forName(crtTblDesc.getOutputFormat());
-      Class<? extends InputFormat>      in_class  = c1;
+      Class<? extends InputFormat> in_class = c1;
       Class<? extends HiveOutputFormat> out_class = c2;
 
       ret.setInputFileFormatClass(in_class);
@@ -206,111 +227,95 @@
   }
 
   /**
-   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the separatorCode.
-   * MetaDataTypedColumnsetSerDe is used because LazySimpleSerDe does not support a table
-   * with a single column "col" with type "array<string>".
+   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the
+   * separatorCode. MetaDataTypedColumnsetSerDe is used because LazySimpleSerDe
+   * does not support a table with a single column "col" with type
+   * "array<string>".
    */
   public static tableDesc getDefaultTableDesc(String separatorCode) {
-    return new tableDesc(
-        MetadataTypedColumnsetSerDe.class,
-        TextInputFormat.class,
-        IgnoreKeyTextOutputFormat.class,
-        Utilities.makeProperties(
-            org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT, separatorCode));
+    return new tableDesc(MetadataTypedColumnsetSerDe.class,
+        TextInputFormat.class, IgnoreKeyTextOutputFormat.class, Utilities
+            .makeProperties(
+                org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT,
+                separatorCode));
   }
 
   /**
    * Generate the table descriptor for reduce key.
    */
-  public static tableDesc getReduceKeyTableDesc(List<FieldSchema> fieldSchemas, String order) {
-    return new tableDesc(
-        BinarySortableSerDe.class,
-        SequenceFileInputFormat.class,
-        SequenceFileOutputFormat.class,
-        Utilities.makeProperties(
-            Constants.LIST_COLUMNS,
-              MetaStoreUtils.getColumnNamesFromFieldSchema(fieldSchemas),
-            Constants.LIST_COLUMN_TYPES,
-              MetaStoreUtils.getColumnTypesFromFieldSchema(fieldSchemas),
-            Constants.SERIALIZATION_SORT_ORDER,
-              order
-        ));
+  public static tableDesc getReduceKeyTableDesc(List<FieldSchema> fieldSchemas,
+      String order) {
+    return new tableDesc(BinarySortableSerDe.class,
+        SequenceFileInputFormat.class, SequenceFileOutputFormat.class,
+        Utilities.makeProperties(Constants.LIST_COLUMNS, MetaStoreUtils
+            .getColumnNamesFromFieldSchema(fieldSchemas),
+            Constants.LIST_COLUMN_TYPES, MetaStoreUtils
+                .getColumnTypesFromFieldSchema(fieldSchemas),
+            Constants.SERIALIZATION_SORT_ORDER, order));
   }
 
   /**
    * Generate the table descriptor for Map-side join key.
    */
   public static tableDesc getMapJoinKeyTableDesc(List<FieldSchema> fieldSchemas) {
-    return new tableDesc(
-        LazyBinarySerDe.class,
-        SequenceFileInputFormat.class,
-        SequenceFileOutputFormat.class,
-        Utilities.makeProperties(
-            "columns", MetaStoreUtils.getColumnNamesFromFieldSchema(fieldSchemas),
-            "columns.types", MetaStoreUtils.getColumnTypesFromFieldSchema(fieldSchemas),
-            Constants.ESCAPE_CHAR, "\\"
-        ));
+    return new tableDesc(LazyBinarySerDe.class, SequenceFileInputFormat.class,
+        SequenceFileOutputFormat.class, Utilities.makeProperties("columns",
+            MetaStoreUtils.getColumnNamesFromFieldSchema(fieldSchemas),
+            "columns.types", MetaStoreUtils
+                .getColumnTypesFromFieldSchema(fieldSchemas),
+            Constants.ESCAPE_CHAR, "\\"));
   }
 
   /**
    * Generate the table descriptor for Map-side join key.
    */
-  public static tableDesc getMapJoinValueTableDesc(List<FieldSchema> fieldSchemas) {
-    return new tableDesc(
-        LazyBinarySerDe.class,
-        SequenceFileInputFormat.class,
-        SequenceFileOutputFormat.class,
-        Utilities.makeProperties(
-            "columns", MetaStoreUtils.getColumnNamesFromFieldSchema(fieldSchemas),
-            "columns.types", MetaStoreUtils.getColumnTypesFromFieldSchema(fieldSchemas),
-            Constants.ESCAPE_CHAR, "\\"
-        ));
+  public static tableDesc getMapJoinValueTableDesc(
+      List<FieldSchema> fieldSchemas) {
+    return new tableDesc(LazyBinarySerDe.class, SequenceFileInputFormat.class,
+        SequenceFileOutputFormat.class, Utilities.makeProperties("columns",
+            MetaStoreUtils.getColumnNamesFromFieldSchema(fieldSchemas),
+            "columns.types", MetaStoreUtils
+                .getColumnTypesFromFieldSchema(fieldSchemas),
+            Constants.ESCAPE_CHAR, "\\"));
   }
 
   /**
    * Generate the table descriptor for intermediate files.
    */
-  public static tableDesc getIntermediateFileTableDesc(List<FieldSchema> fieldSchemas) {
-    return new tableDesc(
-        LazyBinarySerDe.class,
-        SequenceFileInputFormat.class,
-        SequenceFileOutputFormat.class,
-        Utilities.makeProperties(
-            Constants.LIST_COLUMNS,
-              MetaStoreUtils.getColumnNamesFromFieldSchema(fieldSchemas),
-            Constants.LIST_COLUMN_TYPES,
-              MetaStoreUtils.getColumnTypesFromFieldSchema(fieldSchemas),
-            Constants.ESCAPE_CHAR,
-              "\\"
-        ));
+  public static tableDesc getIntermediateFileTableDesc(
+      List<FieldSchema> fieldSchemas) {
+    return new tableDesc(LazyBinarySerDe.class, SequenceFileInputFormat.class,
+        SequenceFileOutputFormat.class, Utilities.makeProperties(
+            Constants.LIST_COLUMNS, MetaStoreUtils
+                .getColumnNamesFromFieldSchema(fieldSchemas),
+            Constants.LIST_COLUMN_TYPES, MetaStoreUtils
+                .getColumnTypesFromFieldSchema(fieldSchemas),
+            Constants.ESCAPE_CHAR, "\\"));
   }
 
   /**
    * Generate the table descriptor for intermediate files.
    */
   public static tableDesc getReduceValueTableDesc(List<FieldSchema> fieldSchemas) {
-    return new tableDesc(
-        LazyBinarySerDe.class,
-        SequenceFileInputFormat.class,
-        SequenceFileOutputFormat.class,
-        Utilities.makeProperties(
-            Constants.LIST_COLUMNS,
-              MetaStoreUtils.getColumnNamesFromFieldSchema(fieldSchemas),
-            Constants.LIST_COLUMN_TYPES,
-              MetaStoreUtils.getColumnTypesFromFieldSchema(fieldSchemas),
-            Constants.ESCAPE_CHAR,
-              "\\"
-        ));
+    return new tableDesc(LazyBinarySerDe.class, SequenceFileInputFormat.class,
+        SequenceFileOutputFormat.class, Utilities.makeProperties(
+            Constants.LIST_COLUMNS, MetaStoreUtils
+                .getColumnNamesFromFieldSchema(fieldSchemas),
+            Constants.LIST_COLUMN_TYPES, MetaStoreUtils
+                .getColumnTypesFromFieldSchema(fieldSchemas),
+            Constants.ESCAPE_CHAR, "\\"));
   }
 
   /**
    * Convert the ColumnList to FieldSchema list.
    */
-  public static List<FieldSchema> getFieldSchemasFromColumnList(List<exprNodeDesc> cols, List<String> outputColumnNames, int start,
+  public static List<FieldSchema> getFieldSchemasFromColumnList(
+      List<exprNodeDesc> cols, List<String> outputColumnNames, int start,
       String fieldPrefix) {
     List<FieldSchema> schemas = new ArrayList<FieldSchema>(cols.size());
-    for (int i=0; i<cols.size(); i++) {
-      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(fieldPrefix + outputColumnNames.get(i+start), cols.get(i).getTypeInfo()));
+    for (int i = 0; i < cols.size(); i++) {
+      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(fieldPrefix
+          + outputColumnNames.get(i + start), cols.get(i).getTypeInfo()));
     }
     return schemas;
   }
@@ -318,11 +323,12 @@
   /**
    * Convert the ColumnList to FieldSchema list.
    */
-  public static List<FieldSchema> getFieldSchemasFromColumnList(List<exprNodeDesc> cols,
-      String fieldPrefix) {
+  public static List<FieldSchema> getFieldSchemasFromColumnList(
+      List<exprNodeDesc> cols, String fieldPrefix) {
     List<FieldSchema> schemas = new ArrayList<FieldSchema>(cols.size());
-    for (int i=0; i<cols.size(); i++) {
-      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(fieldPrefix + i, cols.get(i).getTypeInfo()));
+    for (int i = 0; i < cols.size(); i++) {
+      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(fieldPrefix + i,
+          cols.get(i).getTypeInfo()));
     }
     return schemas;
   }
@@ -330,7 +336,8 @@
   /**
    * Convert the RowSchema to FieldSchema list.
    */
-  public static List<FieldSchema> getFieldSchemasFromRowSchema(RowSchema row, String fieldPrefix) {
+  public static List<FieldSchema> getFieldSchemasFromRowSchema(RowSchema row,
+      String fieldPrefix) {
     Vector<ColumnInfo> c = row.getSignature();
     return getFieldSchemasFromColumnInfo(c, fieldPrefix);
   }
@@ -338,23 +345,26 @@
   /**
    * Convert the ColumnInfo to FieldSchema.
    */
-  public static List<FieldSchema> getFieldSchemasFromColumnInfo(Vector<ColumnInfo> cols, String fieldPrefix) {
-    if ((cols == null) || (cols.size() == 0))
+  public static List<FieldSchema> getFieldSchemasFromColumnInfo(
+      Vector<ColumnInfo> cols, String fieldPrefix) {
+    if ((cols == null) || (cols.size() == 0)) {
       return new ArrayList<FieldSchema>();
+    }
 
     List<FieldSchema> schemas = new ArrayList<FieldSchema>(cols.size());
-    for (int i=0; i<cols.size(); i++) {
+    for (int i = 0; i < cols.size(); i++) {
       String name = cols.get(i).getInternalName();
       if (name.equals(Integer.valueOf(i).toString())) {
         name = fieldPrefix + name;
       }
-      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(name, cols.get(i).getType()));
+      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(name, cols.get(i)
+          .getType()));
     }
     return schemas;
   }
 
   public static List<FieldSchema> sortFieldSchemas(List<FieldSchema> schema) {
-    Collections.sort(schema, new Comparator<FieldSchema>(){
+    Collections.sort(schema, new Comparator<FieldSchema>() {
 
       @Override
       public int compare(FieldSchema o1, FieldSchema o2) {
@@ -367,89 +377,101 @@
 
   /**
    * Create the reduce sink descriptor.
-   * @param keyCols   The columns to be stored in the key
-   * @param valueCols The columns to be stored in the value
-   * @param outputColumnNames The output columns names
-   * @param tag       The tag for this reducesink
-   * @param partitionCols The columns for partitioning.
-   * @param numReducers  The number of reducers, set to -1 for automatic inference
-   *                     based on input data size.
+   * 
+   * @param keyCols
+   *          The columns to be stored in the key
+   * @param valueCols
+   *          The columns to be stored in the value
+   * @param outputColumnNames
+   *          The output columns names
+   * @param tag
+   *          The tag for this reducesink
+   * @param partitionCols
+   *          The columns for partitioning.
+   * @param numReducers
+   *          The number of reducers, set to -1 for automatic inference based on
+   *          input data size.
    * @return The reduceSinkDesc object.
    */
-  public static reduceSinkDesc getReduceSinkDesc(ArrayList<exprNodeDesc> keyCols,
-                                                 ArrayList<exprNodeDesc> valueCols,
-                                                 List<String> outputColumnNames,
-                                                 boolean includeKeyCols,
-                                                 int tag,
-                                                 ArrayList<exprNodeDesc> partitionCols,
-                                                 String order,
-                                                 int numReducers) {
+  public static reduceSinkDesc getReduceSinkDesc(
+      ArrayList<exprNodeDesc> keyCols, ArrayList<exprNodeDesc> valueCols,
+      List<String> outputColumnNames, boolean includeKeyCols, int tag,
+      ArrayList<exprNodeDesc> partitionCols, String order, int numReducers) {
     tableDesc keyTable = null;
     tableDesc valueTable = null;
     ArrayList<String> outputKeyCols = new ArrayList<String>();
     ArrayList<String> outputValCols = new ArrayList<String>();
     if (includeKeyCols) {
-      keyTable = getReduceKeyTableDesc(getFieldSchemasFromColumnList(
-          keyCols, outputColumnNames, 0, ""), order);
+      keyTable = getReduceKeyTableDesc(getFieldSchemasFromColumnList(keyCols,
+          outputColumnNames, 0, ""), order);
       outputKeyCols.addAll(outputColumnNames.subList(0, keyCols.size()));
       valueTable = getReduceValueTableDesc(getFieldSchemasFromColumnList(
           valueCols, outputColumnNames, keyCols.size(), ""));
-      outputValCols.addAll(outputColumnNames.subList(keyCols.size(), outputColumnNames.size()));
+      outputValCols.addAll(outputColumnNames.subList(keyCols.size(),
+          outputColumnNames.size()));
     } else {
-      keyTable = getReduceKeyTableDesc(getFieldSchemasFromColumnList(
-          keyCols, "reducesinkkey"), order);
+      keyTable = getReduceKeyTableDesc(getFieldSchemasFromColumnList(keyCols,
+          "reducesinkkey"), order);
       for (int i = 0; i < keyCols.size(); i++) {
-        outputKeyCols.add("reducesinkkey"+i);
+        outputKeyCols.add("reducesinkkey" + i);
       }
       valueTable = getReduceValueTableDesc(getFieldSchemasFromColumnList(
           valueCols, outputColumnNames, 0, ""));
       outputValCols.addAll(outputColumnNames);
     }
-    return new reduceSinkDesc(keyCols, valueCols, outputKeyCols, outputValCols, tag, partitionCols, numReducers,
-        keyTable,
-        // Revert to DynamicSerDe: getBinaryTableDesc(getFieldSchemasFromColumnList(valueCols, "reducesinkvalue")));
+    return new reduceSinkDesc(keyCols, valueCols, outputKeyCols, outputValCols,
+        tag, partitionCols, numReducers, keyTable,
+        // Revert to DynamicSerDe:
+        // getBinaryTableDesc(getFieldSchemasFromColumnList(valueCols,
+        // "reducesinkvalue")));
         valueTable);
   }
 
   /**
    * Create the reduce sink descriptor.
-   * @param keyCols   The columns to be stored in the key
-   * @param valueCols The columns to be stored in the value
-   * @param outputColumnNames The output columns names
-   * @param tag       The tag for this reducesink
-   * @param numPartitionFields  The first numPartitionFields of keyCols will be partition columns.
-   *                  If numPartitionFields=-1, then partition randomly.
-   * @param numReducers  The number of reducers, set to -1 for automatic inference
-   *                     based on input data size.
+   * 
+   * @param keyCols
+   *          The columns to be stored in the key
+   * @param valueCols
+   *          The columns to be stored in the value
+   * @param outputColumnNames
+   *          The output columns names
+   * @param tag
+   *          The tag for this reducesink
+   * @param numPartitionFields
+   *          The first numPartitionFields of keyCols will be partition columns.
+   *          If numPartitionFields=-1, then partition randomly.
+   * @param numReducers
+   *          The number of reducers, set to -1 for automatic inference based on
+   *          input data size.
    * @return The reduceSinkDesc object.
    */
   public static reduceSinkDesc getReduceSinkDesc(
       ArrayList<exprNodeDesc> keyCols, ArrayList<exprNodeDesc> valueCols,
-      List<String> outputColumnNames, boolean includeKey, int tag, int numPartitionFields,
-      int numReducers) {
+      List<String> outputColumnNames, boolean includeKey, int tag,
+      int numPartitionFields, int numReducers) {
     ArrayList<exprNodeDesc> partitionCols = null;
 
     if (numPartitionFields >= keyCols.size()) {
       partitionCols = keyCols;
     } else if (numPartitionFields >= 0) {
       partitionCols = new ArrayList<exprNodeDesc>(numPartitionFields);
-      for (int i=0; i<numPartitionFields; i++) {
+      for (int i = 0; i < numPartitionFields; i++) {
         partitionCols.add(keyCols.get(i));
       }
     } else {
       // numPartitionFields = -1 means random partitioning
       partitionCols = new ArrayList<exprNodeDesc>(1);
-      partitionCols.add(TypeCheckProcFactory.DefaultExprProcessor.getFuncExprNodeDesc("rand"));
+      partitionCols.add(TypeCheckProcFactory.DefaultExprProcessor
+          .getFuncExprNodeDesc("rand"));
     }
 
     StringBuilder order = new StringBuilder();
-    for (int i=0; i<keyCols.size(); i++) {
+    for (int i = 0; i < keyCols.size(); i++) {
       order.append("+");
     }
-    return getReduceSinkDesc(keyCols, valueCols, outputColumnNames, includeKey, tag, partitionCols, order.toString(),
-         numReducers);
+    return getReduceSinkDesc(keyCols, valueCols, outputColumnNames, includeKey,
+        tag, partitionCols, order.toString(), numReducers);
   }
 
-
 }
-

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/aggregationDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/aggregationDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/aggregationDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/aggregationDesc.java Thu Jan 21 10:37:58 2010
@@ -18,75 +18,80 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
-import java.io.Serializable;
-
-import org.apache.hadoop.hive.ql.exec.FunctionInfo;
-import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
-import org.apache.hadoop.hive.ql.exec.UDAFEvaluator;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
-import org.apache.hadoop.util.ReflectionUtils;
 
 public class aggregationDesc implements java.io.Serializable {
   private static final long serialVersionUID = 1L;
   private String genericUDAFName;
-  
+
   /**
    * In case genericUDAFEvaluator is Serializable, we will serialize the object.
    * 
-   * In case genericUDAFEvaluator does not implement Serializable, Java will remember the
-   * class of genericUDAFEvaluator and creates a new instance when deserialized.  This is
-   * exactly what we want.
+   * In case genericUDAFEvaluator does not implement Serializable, Java will
+   * remember the class of genericUDAFEvaluator and creates a new instance when
+   * deserialized. This is exactly what we want.
    */
   private GenericUDAFEvaluator genericUDAFEvaluator;
   private java.util.ArrayList<exprNodeDesc> parameters;
   private boolean distinct;
   private GenericUDAFEvaluator.Mode mode;
-  public aggregationDesc() {}
-  public aggregationDesc(
-    final String genericUDAFName,
-    final GenericUDAFEvaluator genericUDAFEvaluator,
-    final java.util.ArrayList<exprNodeDesc> parameters,
-    final boolean distinct,
-    final GenericUDAFEvaluator.Mode mode) {
+
+  public aggregationDesc() {
+  }
+
+  public aggregationDesc(final String genericUDAFName,
+      final GenericUDAFEvaluator genericUDAFEvaluator,
+      final java.util.ArrayList<exprNodeDesc> parameters,
+      final boolean distinct, final GenericUDAFEvaluator.Mode mode) {
     this.genericUDAFName = genericUDAFName;
     this.genericUDAFEvaluator = genericUDAFEvaluator;
     this.parameters = parameters;
     this.distinct = distinct;
     this.mode = mode;
   }
+
   public void setGenericUDAFName(final String genericUDAFName) {
     this.genericUDAFName = genericUDAFName;
   }
+
   public String getGenericUDAFName() {
     return genericUDAFName;
   }
-  public void setGenericUDAFEvaluator(final GenericUDAFEvaluator genericUDAFEvaluator) {
+
+  public void setGenericUDAFEvaluator(
+      final GenericUDAFEvaluator genericUDAFEvaluator) {
     this.genericUDAFEvaluator = genericUDAFEvaluator;
   }
+
   public GenericUDAFEvaluator getGenericUDAFEvaluator() {
     return genericUDAFEvaluator;
   }
+
   public java.util.ArrayList<exprNodeDesc> getParameters() {
-    return this.parameters;
+    return parameters;
   }
+
   public void setParameters(final java.util.ArrayList<exprNodeDesc> parameters) {
-    this.parameters=parameters;
+    this.parameters = parameters;
   }
+
   public boolean getDistinct() {
-    return this.distinct;
+    return distinct;
   }
+
   public void setDistinct(final boolean distinct) {
     this.distinct = distinct;
   }
+
   public void setMode(final GenericUDAFEvaluator.Mode mode) {
     this.mode = mode;
   }
-  
+
   public GenericUDAFEvaluator.Mode getMode() {
     return mode;
   }
-  
-  @explain(displayName="expr")
+
+  @explain(displayName = "expr")
   public String getExprString() {
     StringBuilder sb = new StringBuilder();
     sb.append(genericUDAFName);
@@ -95,7 +100,7 @@
       sb.append("DISTINCT ");
     }
     boolean first = true;
-    for(exprNodeDesc exp: parameters) {
+    for (exprNodeDesc exp : parameters) {
       if (first) {
         first = false;
       } else {

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/alterTableDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/alterTableDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/alterTableDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/alterTableDesc.java Thu Jan 21 10:37:58 2010
@@ -26,54 +26,61 @@
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 
-@explain(displayName="Alter Table")
-public class alterTableDesc extends ddlDesc implements Serializable 
-{
+@explain(displayName = "Alter Table")
+public class alterTableDesc extends ddlDesc implements Serializable {
   private static final long serialVersionUID = 1L;
-  public static enum alterTableTypes {RENAME, ADDCOLS, REPLACECOLS, ADDPROPS, ADDSERDE, ADDSERDEPROPS, ADDFILEFORMAT, ADDCLUSTERSORTCOLUMN, RENAMECOLUMN};
-    
-  alterTableTypes      op;
-  String               oldName;
-  String               newName;
-  List<FieldSchema>    newCols;
-  String               serdeName;
-  Map<String, String>  props;
-  String	             inputFormat;
-	String               outputFormat;
-	int                  numberBuckets;
-	List<String>         bucketColumns;
-	List<Order>          sortColumns;
-
-
-  String              oldColName;
-  String              newColName;
-  String              newColType;
-  String              newColComment;
-  boolean             first;
-  String              afterCol;
-  
-  /**
-   * @param tblName table name
-   * @param oldColName  old column name
-   * @param newColName  new column name
-   * @param newComment 
-   * @param newType 
+
+  public static enum alterTableTypes {
+    RENAME, ADDCOLS, REPLACECOLS, ADDPROPS, ADDSERDE, ADDSERDEPROPS, ADDFILEFORMAT, ADDCLUSTERSORTCOLUMN, RENAMECOLUMN
+  };
+
+  alterTableTypes op;
+  String oldName;
+  String newName;
+  List<FieldSchema> newCols;
+  String serdeName;
+  Map<String, String> props;
+  String inputFormat;
+  String outputFormat;
+  int numberBuckets;
+  List<String> bucketColumns;
+  List<Order> sortColumns;
+
+  String oldColName;
+  String newColName;
+  String newColType;
+  String newColComment;
+  boolean first;
+  String afterCol;
+
+  /**
+   * @param tblName
+   *          table name
+   * @param oldColName
+   *          old column name
+   * @param newColName
+   *          new column name
+   * @param newComment
+   * @param newType
    */
-  public alterTableDesc(String tblName, String oldColName, String newColName, String newType, String newComment, boolean first, String afterCol) {
+  public alterTableDesc(String tblName, String oldColName, String newColName,
+      String newType, String newComment, boolean first, String afterCol) {
     super();
-    this.oldName = tblName;
+    oldName = tblName;
     this.oldColName = oldColName;
     this.newColName = newColName;
-    this.newColType = newType;
-    this.newColComment = newComment;
+    newColType = newType;
+    newColComment = newComment;
     this.first = first;
     this.afterCol = afterCol;
-    this.op = alterTableTypes.RENAMECOLUMN;
+    op = alterTableTypes.RENAMECOLUMN;
   }
 
   /**
-   * @param oldName old name of the table
-   * @param newName new name of the table
+   * @param oldName
+   *          old name of the table
+   * @param newName
+   *          new name of the table
    */
   public alterTableDesc(String oldName, String newName) {
     op = alterTableTypes.RENAME;
@@ -82,56 +89,65 @@
   }
 
   /**
-   * @param name name of the table
-   * @param newCols new columns to be added
-   */
-  public alterTableDesc(String name, List<FieldSchema> newCols, alterTableTypes alterType) {
-    this.op = alterType;
-    this.oldName = name;
+   * @param name
+   *          name of the table
+   * @param newCols
+   *          new columns to be added
+   */
+  public alterTableDesc(String name, List<FieldSchema> newCols,
+      alterTableTypes alterType) {
+    op = alterType;
+    oldName = name;
     this.newCols = newCols;
   }
-  
+
   /**
-   * @param alterType type of alter op
+   * @param alterType
+   *          type of alter op
    */
   public alterTableDesc(alterTableTypes alterType) {
-    this.op = alterType;
+    op = alterType;
   }
 
   /**
    * 
-   * @param name name of the table
-   * @param inputFormat new table input format
-   * @param outputFormat new table output format 
-   */
-  public alterTableDesc(String name, String inputFormat, String outputFormat, String serdeName) {
-	  super();
-	  this.op = alterTableTypes.ADDFILEFORMAT;
-	  this.oldName = name;
-	  this.inputFormat = inputFormat;
-	  this.outputFormat = outputFormat;
-	  this.serdeName = serdeName;
+   * @param name
+   *          name of the table
+   * @param inputFormat
+   *          new table input format
+   * @param outputFormat
+   *          new table output format
+   */
+  public alterTableDesc(String name, String inputFormat, String outputFormat,
+      String serdeName) {
+    super();
+    op = alterTableTypes.ADDFILEFORMAT;
+    oldName = name;
+    this.inputFormat = inputFormat;
+    this.outputFormat = outputFormat;
+    this.serdeName = serdeName;
   }
-  
+
   public alterTableDesc(String tableName, int numBuckets,
       List<String> bucketCols, List<Order> sortCols) {
-  	this.oldName = tableName;
-  	this.op = alterTableTypes.ADDCLUSTERSORTCOLUMN;
-  	this.numberBuckets = numBuckets;
-  	this.bucketColumns = bucketCols;
-  	this.sortColumns = sortCols;
+    oldName = tableName;
+    op = alterTableTypes.ADDCLUSTERSORTCOLUMN;
+    numberBuckets = numBuckets;
+    bucketColumns = bucketCols;
+    sortColumns = sortCols;
   }
 
-	/**
+  /**
    * @return the old name of the table
    */
-  @explain(displayName="old name")
+  @explain(displayName = "old name")
   public String getOldName() {
     return oldName;
   }
 
   /**
-   * @param oldName the oldName to set
+   * @param oldName
+   *          the oldName to set
    */
   public void setOldName(String oldName) {
     this.oldName = oldName;
@@ -140,13 +156,14 @@
   /**
    * @return the newName
    */
-  @explain(displayName="new name")
+  @explain(displayName = "new name")
   public String getNewName() {
     return newName;
   }
 
   /**
-   * @param newName the newName to set
+   * @param newName
+   *          the newName to set
    */
   public void setNewName(String newName) {
     this.newName = newName;
@@ -159,9 +176,9 @@
     return op;
   }
 
-  @explain(displayName="type")
+  @explain(displayName = "type")
   public String getAlterTableTypeString() {
-    switch(op) {
+    switch (op) {
     case RENAME:
       return "rename";
     case ADDCOLS:
@@ -169,11 +186,13 @@
     case REPLACECOLS:
       return "replace columns";
     }
-    
+
     return "unknown";
   }
+
   /**
-   * @param op the op to set
+   * @param op
+   *          the op to set
    */
   public void setOp(alterTableTypes op) {
     this.op = op;
@@ -186,12 +205,14 @@
     return newCols;
   }
 
-  @explain(displayName="new columns")
+  @explain(displayName = "new columns")
   public List<String> getNewColsString() {
     return Utilities.getFieldSchemaString(getNewCols());
   }
+
   /**
-   * @param newCols the newCols to set
+   * @param newCols
+   *          the newCols to set
    */
   public void setNewCols(List<FieldSchema> newCols) {
     this.newCols = newCols;
@@ -200,13 +221,14 @@
   /**
    * @return the serdeName
    */
-  @explain(displayName="deserializer library")
+  @explain(displayName = "deserializer library")
   public String getSerdeName() {
     return serdeName;
   }
 
   /**
-   * @param serdeName the serdeName to set
+   * @param serdeName
+   *          the serdeName to set
    */
   public void setSerdeName(String serdeName) {
     this.serdeName = serdeName;
@@ -215,99 +237,106 @@
   /**
    * @return the props
    */
-  @explain(displayName="properties")
+  @explain(displayName = "properties")
   public Map<String, String> getProps() {
     return props;
   }
 
   /**
-   * @param props the props to set
+   * @param props
+   *          the props to set
    */
   public void setProps(Map<String, String> props) {
     this.props = props;
   }
-  
+
   /**
    * @return the input format
    */
-  @explain(displayName="input format")
-	public String getInputFormat() {
-  	return inputFormat;
+  @explain(displayName = "input format")
+  public String getInputFormat() {
+    return inputFormat;
   }
 
   /**
-   * @param inputFormat the input format to set
+   * @param inputFormat
+   *          the input format to set
    */
-	public void setInputFormat(String inputFormat) {
-  	this.inputFormat = inputFormat;
+  public void setInputFormat(String inputFormat) {
+    this.inputFormat = inputFormat;
   }
 
   /**
    * @return the output format
    */
-  @explain(displayName="output format")
-	public String getOutputFormat() {
-  	return outputFormat;
+  @explain(displayName = "output format")
+  public String getOutputFormat() {
+    return outputFormat;
   }
 
   /**
-   * @param outputFormat the output format to set
+   * @param outputFormat
+   *          the output format to set
    */
-	public void setOutputFormat(String outputFormat) {
-  	this.outputFormat = outputFormat;
+  public void setOutputFormat(String outputFormat) {
+    this.outputFormat = outputFormat;
   }
 
-	/**
-	 * @return the number of buckets
-	 */
-	public int getNumberBuckets() {
-  	return numberBuckets;
+  /**
+   * @return the number of buckets
+   */
+  public int getNumberBuckets() {
+    return numberBuckets;
   }
 
-	/**
-	 * @param numberBuckets the number of buckets to set
-	 */
-	public void setNumberBuckets(int numberBuckets) {
-  	this.numberBuckets = numberBuckets;
+  /**
+   * @param numberBuckets
+   *          the number of buckets to set
+   */
+  public void setNumberBuckets(int numberBuckets) {
+    this.numberBuckets = numberBuckets;
   }
 
-	/**
-	 * @return the bucket columns
-	 */
-	public List<String> getBucketColumns() {
-  	return bucketColumns;
+  /**
+   * @return the bucket columns
+   */
+  public List<String> getBucketColumns() {
+    return bucketColumns;
   }
 
-	/**
-	 * @param bucketColumns the bucket columns to set
-	 */
-	public void setBucketColumns(List<String> bucketColumns) {
-  	this.bucketColumns = bucketColumns;
+  /**
+   * @param bucketColumns
+   *          the bucket columns to set
+   */
+  public void setBucketColumns(List<String> bucketColumns) {
+    this.bucketColumns = bucketColumns;
   }
 
-	/**
-	 * @return the sort columns
-	 */
-	public List<Order> getSortColumns() {
-  	return sortColumns;
+  /**
+   * @return the sort columns
+   */
+  public List<Order> getSortColumns() {
+    return sortColumns;
   }
 
-	/**
-	 * @param sortColumns the sort columns to set
-	 */
-	public void setSortColumns(List<Order> sortColumns) {
-  	this.sortColumns = sortColumns;
+  /**
+   * @param sortColumns
+   *          the sort columns to set
+   */
+  public void setSortColumns(List<Order> sortColumns) {
+    this.sortColumns = sortColumns;
   }
 
-/**
- * @return old column name
- */
+  /**
+   * @return old column name
+   */
   public String getOldColName() {
     return oldColName;
   }
 
   /**
-   * @param oldColName the old column name
+   * @param oldColName
+   *          the old column name
    */
   public void setOldColName(String oldColName) {
     this.oldColName = oldColName;
@@ -321,7 +350,8 @@
   }
 
   /**
-   * @param newColName the new column name
+   * @param newColName
+   *          the new column name
    */
   public void setNewColName(String newColName) {
     this.newColName = newColName;
@@ -335,10 +365,11 @@
   }
 
   /**
-   * @param newType new column's type
+   * @param newType
+   *          new column's type
    */
   public void setNewColType(String newType) {
-    this.newColType = newType;
+    newColType = newType;
   }
 
   /**
@@ -349,10 +380,11 @@
   }
 
   /**
-   * @param newComment new column's comment
+   * @param newComment
+   *          new column's comment
    */
   public void setNewColComment(String newComment) {
-    this.newColComment = newComment;
+    newColComment = newComment;
   }
 
   /**
@@ -363,7 +395,8 @@
   }
 
   /**
-   * @param first set the column to position 0
+   * @param first
+   *          set the column to position 0
    */
   public void setFirst(boolean first) {
     this.first = first;
@@ -377,7 +410,8 @@
   }
 
   /**
-   * @param afterCol set the column's after position
+   * @param afterCol
+   *          set the column's after position
    */
   public void setAfterCol(String afterCol) {
     this.afterCol = afterCol;

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/collectDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/collectDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/collectDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/collectDesc.java Thu Jan 21 10:37:58 2010
@@ -20,19 +20,23 @@
 
 import java.io.Serializable;
 
-@explain(displayName="Collect")
+@explain(displayName = "Collect")
 public class collectDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   Integer bufferSize;
-  public collectDesc() {}
-  public collectDesc(
-    final Integer bufferSize) {
+
+  public collectDesc() {
+  }
+
+  public collectDesc(final Integer bufferSize) {
     this.bufferSize = bufferSize;
   }
+
   public Integer getBufferSize() {
-    return this.bufferSize;
+    return bufferSize;
   }
+
   public void setBufferSize(Integer bufferSize) {
-    this.bufferSize=bufferSize;
+    this.bufferSize = bufferSize;
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/copyWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/copyWork.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/copyWork.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/copyWork.java Thu Jan 21 10:37:58 2010
@@ -20,33 +20,35 @@
 
 import java.io.Serializable;
 
-@explain(displayName="Copy")
+@explain(displayName = "Copy")
 public class copyWork implements Serializable {
   private static final long serialVersionUID = 1L;
   private String fromPath;
   private String toPath;
 
-  public copyWork() { }
-  public copyWork(
-    final String fromPath,
-    final String toPath) {
+  public copyWork() {
+  }
+
+  public copyWork(final String fromPath, final String toPath) {
     this.fromPath = fromPath;
     this.toPath = toPath;
   }
-  
-  @explain(displayName="source")
+
+  @explain(displayName = "source")
   public String getFromPath() {
-    return this.fromPath;
+    return fromPath;
   }
+
   public void setFromPath(final String fromPath) {
     this.fromPath = fromPath;
   }
-  
-  @explain(displayName="destination")
+
+  @explain(displayName = "destination")
   public String getToPath() {
-    return this.toPath;
+    return toPath;
   }
+
   public void setToPath(final String toPath) {
-    this.toPath=toPath;
+    this.toPath = toPath;
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createFunctionDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createFunctionDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createFunctionDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createFunctionDesc.java Thu Jan 21 10:37:58 2010
@@ -20,19 +20,19 @@
 
 import java.io.Serializable;
 
-@explain(displayName="Create Function")
+@explain(displayName = "Create Function")
 public class createFunctionDesc implements Serializable {
   private static final long serialVersionUID = 1L;
-  
+
   private String functionName;
   private String className;
-  
+
   public createFunctionDesc(String functionName, String className) {
     this.functionName = functionName;
     this.className = className;
   }
 
-  @explain(displayName="name")
+  @explain(displayName = "name")
   public String getFunctionName() {
     return functionName;
   }
@@ -41,7 +41,7 @@
     this.functionName = functionName;
   }
 
-  @explain(displayName="class")
+  @explain(displayName = "class")
   public String getClassName() {
     return className;
   }
@@ -49,5 +49,5 @@
   public void setClassName(String className) {
     this.className = className;
   }
-  
+
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java Thu Jan 21 10:37:58 2010
@@ -26,62 +26,58 @@
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 
-@explain(displayName="Create Table")
-public class createTableDesc extends ddlDesc implements Serializable 
-{
+@explain(displayName = "Create Table")
+public class createTableDesc extends ddlDesc implements Serializable {
   private static final long serialVersionUID = 1L;
-  String              tableName;
-  boolean             isExternal;
-  List<FieldSchema>   cols;
-  List<FieldSchema>   partCols;
-  List<String>        bucketCols;
-  List<Order>         sortCols;
-  int                 numBuckets;
-  String              fieldDelim;
-  String              fieldEscape;
-  String              collItemDelim;
-  String              mapKeyDelim;
-  String              lineDelim;
-  String              comment;
-  String              inputFormat;
-  String              outputFormat;
-  String              location;
-  String              serName;
+  String tableName;
+  boolean isExternal;
+  List<FieldSchema> cols;
+  List<FieldSchema> partCols;
+  List<String> bucketCols;
+  List<Order> sortCols;
+  int numBuckets;
+  String fieldDelim;
+  String fieldEscape;
+  String collItemDelim;
+  String mapKeyDelim;
+  String lineDelim;
+  String comment;
+  String inputFormat;
+  String outputFormat;
+  String location;
+  String serName;
   Map<String, String> mapProp;
-  boolean             ifNotExists;
-  
-  public createTableDesc(String tableName, boolean isExternal, 
-                         List<FieldSchema> cols, List<FieldSchema> partCols,
-                         List<String> bucketCols, List<Order> sortCols, 
-                         int numBuckets, String fieldDelim, 
-                         String fieldEscape,
-                         String collItemDelim,
-                         String mapKeyDelim, String lineDelim, 
-                         String comment, String inputFormat, String outputFormat,
-                         String location, String serName, Map<String, String> mapProp,
-                         boolean ifNotExists) {
-    this.tableName       = tableName;
-    this.isExternal      = isExternal;
-    this.bucketCols      = bucketCols;
-    this.sortCols        = sortCols;
-    this.collItemDelim   = collItemDelim;
-    this.cols            = cols;
-    this.comment         = comment;
-    this.fieldDelim      = fieldDelim;
-    this.fieldEscape     = fieldEscape;
-    this.inputFormat     = inputFormat;
-    this.outputFormat    = outputFormat;
-    this.lineDelim       = lineDelim;
-    this.location        = location;
-    this.mapKeyDelim     = mapKeyDelim;
-    this.numBuckets      = numBuckets;
-    this.partCols        = partCols;
-    this.serName         = serName;
-    this.mapProp         = mapProp;
-    this.ifNotExists     = ifNotExists;
+  boolean ifNotExists;
+
+  public createTableDesc(String tableName, boolean isExternal,
+      List<FieldSchema> cols, List<FieldSchema> partCols,
+      List<String> bucketCols, List<Order> sortCols, int numBuckets,
+      String fieldDelim, String fieldEscape, String collItemDelim,
+      String mapKeyDelim, String lineDelim, String comment, String inputFormat,
+      String outputFormat, String location, String serName,
+      Map<String, String> mapProp, boolean ifNotExists) {
+    this.tableName = tableName;
+    this.isExternal = isExternal;
+    this.bucketCols = bucketCols;
+    this.sortCols = sortCols;
+    this.collItemDelim = collItemDelim;
+    this.cols = cols;
+    this.comment = comment;
+    this.fieldDelim = fieldDelim;
+    this.fieldEscape = fieldEscape;
+    this.inputFormat = inputFormat;
+    this.outputFormat = outputFormat;
+    this.lineDelim = lineDelim;
+    this.location = location;
+    this.mapKeyDelim = mapKeyDelim;
+    this.numBuckets = numBuckets;
+    this.partCols = partCols;
+    this.serName = serName;
+    this.mapProp = mapProp;
+    this.ifNotExists = ifNotExists;
   }
 
-  @explain(displayName="if not exists")
+  @explain(displayName = "if not exists")
   public boolean getIfNotExists() {
     return ifNotExists;
   }
@@ -90,7 +86,7 @@
     this.ifNotExists = ifNotExists;
   }
 
-  @explain(displayName="name")
+  @explain(displayName = "name")
   public String getTableName() {
     return tableName;
   }
@@ -103,11 +99,11 @@
     return cols;
   }
 
-  @explain(displayName="columns")
+  @explain(displayName = "columns")
   public List<String> getColsString() {
     return Utilities.getFieldSchemaString(getCols());
   }
- 
+
   public void setCols(List<FieldSchema> cols) {
     this.cols = cols;
   }
@@ -116,7 +112,7 @@
     return partCols;
   }
 
-  @explain(displayName="partition columns")
+  @explain(displayName = "partition columns")
   public List<String> getPartColsString() {
     return Utilities.getFieldSchemaString(getPartCols());
   }
@@ -125,7 +121,7 @@
     this.partCols = partCols;
   }
 
-  @explain(displayName="bucket columns")
+  @explain(displayName = "bucket columns")
   public List<String> getBucketCols() {
     return bucketCols;
   }
@@ -134,7 +130,7 @@
     this.bucketCols = bucketCols;
   }
 
-  @explain(displayName="# buckets")
+  @explain(displayName = "# buckets")
   public int getNumBuckets() {
     return numBuckets;
   }
@@ -143,7 +139,7 @@
     this.numBuckets = numBuckets;
   }
 
-  @explain(displayName="field delimiter")
+  @explain(displayName = "field delimiter")
   public String getFieldDelim() {
     return fieldDelim;
   }
@@ -152,7 +148,7 @@
     this.fieldDelim = fieldDelim;
   }
 
-  @explain(displayName="field escape")
+  @explain(displayName = "field escape")
   public String getFieldEscape() {
     return fieldEscape;
   }
@@ -161,7 +157,7 @@
     this.fieldEscape = fieldEscape;
   }
 
-  @explain(displayName="collection delimiter")
+  @explain(displayName = "collection delimiter")
   public String getCollItemDelim() {
     return collItemDelim;
   }
@@ -170,7 +166,7 @@
     this.collItemDelim = collItemDelim;
   }
 
-  @explain(displayName="map key delimiter")
+  @explain(displayName = "map key delimiter")
   public String getMapKeyDelim() {
     return mapKeyDelim;
   }
@@ -179,7 +175,7 @@
     this.mapKeyDelim = mapKeyDelim;
   }
 
-  @explain(displayName="line delimiter")
+  @explain(displayName = "line delimiter")
   public String getLineDelim() {
     return lineDelim;
   }
@@ -188,7 +184,7 @@
     this.lineDelim = lineDelim;
   }
 
-  @explain(displayName="comment")
+  @explain(displayName = "comment")
   public String getComment() {
     return comment;
   }
@@ -197,7 +193,7 @@
     this.comment = comment;
   }
 
-  @explain(displayName="input format")
+  @explain(displayName = "input format")
   public String getInputFormat() {
     return inputFormat;
   }
@@ -206,7 +202,7 @@
     this.inputFormat = inputFormat;
   }
 
-  @explain(displayName="output format")
+  @explain(displayName = "output format")
   public String getOutputFormat() {
     return outputFormat;
   }
@@ -215,7 +211,7 @@
     this.outputFormat = outputFormat;
   }
 
-  @explain(displayName="location")
+  @explain(displayName = "location")
   public String getLocation() {
     return location;
   }
@@ -224,7 +220,7 @@
     this.location = location;
   }
 
-  @explain(displayName="isExternal")
+  @explain(displayName = "isExternal")
   public boolean isExternal() {
     return isExternal;
   }
@@ -236,46 +232,49 @@
   /**
    * @return the sortCols
    */
-  @explain(displayName="sort columns")
+  @explain(displayName = "sort columns")
   public List<Order> getSortCols() {
     return sortCols;
   }
 
   /**
-   * @param sortCols the sortCols to set
+   * @param sortCols
+   *          the sortCols to set
    */
   public void setSortCols(List<Order> sortCols) {
     this.sortCols = sortCols;
   }
 
-	/**
-	 * @return the serDeName
-	 */
-  @explain(displayName="serde name")
-	public String getSerName() {
-		return serName;
-	}
-
-	/**
-	 * @param serName the serName to set
-	 */
-	public void setSerName(String serName) {
-		this.serName = serName;
-	}
-
-	/**
-	 * @return the serDe properties
-	 */
-  @explain(displayName="serde properties")
-  public Map<String,String> getMapProp() {
-		return mapProp;
-	}
-
-	/**
-	 * @param mapProp the map properties to set
-	 */
-	public void setMapProp(Map<String,String> mapProp) {
-		this.mapProp = mapProp;
-	}
-  
+  /**
+   * @return the serDeName
+   */
+  @explain(displayName = "serde name")
+  public String getSerName() {
+    return serName;
+  }
+
+  /**
+   * @param serName
+   *          the serName to set
+   */
+  public void setSerName(String serName) {
+    this.serName = serName;
+  }
+
+  /**
+   * @return the serDe properties
+   */
+  @explain(displayName = "serde properties")
+  public Map<String, String> getMapProp() {
+    return mapProp;
+  }
+
+  /**
+   * @param mapProp
+   *          the map properties to set
+   */
+  public void setMapProp(Map<String, String> mapProp) {
+    this.mapProp = mapProp;
+  }
+
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableLikeDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableLikeDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableLikeDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableLikeDesc.java Thu Jan 21 10:37:58 2010
@@ -20,28 +20,25 @@
 
 import java.io.Serializable;
 
-@explain(displayName="Create Table")
-public class createTableLikeDesc extends ddlDesc implements Serializable 
-{
+@explain(displayName = "Create Table")
+public class createTableLikeDesc extends ddlDesc implements Serializable {
   private static final long serialVersionUID = 1L;
-  String              tableName;
-  boolean             isExternal;
-  String              location;
-  boolean             ifNotExists;
-  String              likeTableName;
-  
-  public createTableLikeDesc(String tableName, boolean isExternal, 
-                         String location,
-                         boolean ifNotExists,
-                         String likeTableName) {
-    this.tableName       = tableName;
-    this.isExternal      = isExternal;
-    this.location        = location;
-    this.ifNotExists     = ifNotExists;
-    this.likeTableName   = likeTableName;
+  String tableName;
+  boolean isExternal;
+  String location;
+  boolean ifNotExists;
+  String likeTableName;
+
+  public createTableLikeDesc(String tableName, boolean isExternal,
+      String location, boolean ifNotExists, String likeTableName) {
+    this.tableName = tableName;
+    this.isExternal = isExternal;
+    this.location = location;
+    this.ifNotExists = ifNotExists;
+    this.likeTableName = likeTableName;
   }
 
-  @explain(displayName="if not exists")
+  @explain(displayName = "if not exists")
   public boolean getIfNotExists() {
     return ifNotExists;
   }
@@ -50,7 +47,7 @@
     this.ifNotExists = ifNotExists;
   }
 
-  @explain(displayName="name")
+  @explain(displayName = "name")
   public String getTableName() {
     return tableName;
   }
@@ -59,7 +56,7 @@
     this.tableName = tableName;
   }
 
-  @explain(displayName="location")
+  @explain(displayName = "location")
   public String getLocation() {
     return location;
   }
@@ -68,7 +65,7 @@
     this.location = location;
   }
 
-  @explain(displayName="isExternal")
+  @explain(displayName = "isExternal")
   public boolean isExternal() {
     return isExternal;
   }
@@ -77,7 +74,7 @@
     this.isExternal = isExternal;
   }
 
-  @explain(displayName="like")
+  @explain(displayName = "like")
   public String getLikeTableName() {
     return likeTableName;
   }
@@ -86,5 +83,4 @@
     this.likeTableName = likeTableName;
   }
 
-  
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createViewDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createViewDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createViewDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createViewDesc.java Thu Jan 21 10:37:58 2010
@@ -18,14 +18,13 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-
+import java.io.Serializable;
 import java.util.List;
 
-import java.io.Serializable;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 
-@explain(displayName="Create View")
+@explain(displayName = "Create View")
 public class createViewDesc implements Serializable {
   private static final long serialVersionUID = 1L;
 
@@ -36,16 +35,15 @@
   private String comment;
   private boolean ifNotExists;
 
-  public createViewDesc(
-    String viewName, List<FieldSchema> schema, String comment,
-    boolean ifNotExists) {
+  public createViewDesc(String viewName, List<FieldSchema> schema,
+      String comment, boolean ifNotExists) {
     this.viewName = viewName;
     this.schema = schema;
     this.comment = comment;
     this.ifNotExists = ifNotExists;
   }
 
-  @explain(displayName="name")
+  @explain(displayName = "name")
   public String getViewName() {
     return viewName;
   }
@@ -54,7 +52,7 @@
     this.viewName = viewName;
   }
 
-  @explain(displayName="original text")
+  @explain(displayName = "original text")
   public String getViewOriginalText() {
     return originalText;
   }
@@ -63,7 +61,7 @@
     this.originalText = originalText;
   }
 
-  @explain(displayName="expanded text")
+  @explain(displayName = "expanded text")
   public String getViewExpandedText() {
     return expandedText;
   }
@@ -72,11 +70,11 @@
     this.expandedText = expandedText;
   }
 
-  @explain(displayName="columns")
+  @explain(displayName = "columns")
   public List<String> getSchemaString() {
     return Utilities.getFieldSchemaString(schema);
   }
-  
+
   public List<FieldSchema> getSchema() {
     return schema;
   }
@@ -85,7 +83,7 @@
     this.schema = schema;
   }
 
-  @explain(displayName="comment")
+  @explain(displayName = "comment")
   public String getComment() {
     return comment;
   }
@@ -94,7 +92,7 @@
     this.comment = comment;
   }
 
-  @explain(displayName="if not exists")
+  @explain(displayName = "if not exists")
   public boolean getIfNotExists() {
     return ifNotExists;
   }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ddlDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ddlDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ddlDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ddlDesc.java Thu Jan 21 10:37:58 2010
@@ -20,7 +20,6 @@
 
 import java.io.Serializable;
 
-public abstract class ddlDesc implements Serializable 
-{
-	private static final long serialVersionUID = 1L;
+public abstract class ddlDesc implements Serializable {
+  private static final long serialVersionUID = 1L;
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/descFunctionDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/descFunctionDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/descFunctionDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/descFunctionDesc.java Thu Jan 21 10:37:58 2010
@@ -19,16 +19,16 @@
 package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
+
 import org.apache.hadoop.fs.Path;
 
-@explain(displayName="Describe Function")
-public class descFunctionDesc extends ddlDesc implements Serializable
-{
+@explain(displayName = "Describe Function")
+public class descFunctionDesc extends ddlDesc implements Serializable {
   private static final long serialVersionUID = 1L;
-  String     name;
-  Path       resFile;
-  boolean    isExtended;
-  
+  String name;
+  Path resFile;
+  boolean isExtended;
+
   public boolean isExtended() {
     return isExtended;
   }
@@ -36,7 +36,7 @@
   public void setExtended(boolean isExtended) {
     this.isExtended = isExtended;
   }
-  
+
   /**
    * table name for the result of show tables
    */
@@ -53,7 +53,7 @@
   public String getSchema() {
     return schema;
   }
-  
+
   /**
    * @param resFile
    */
@@ -63,7 +63,8 @@
   }
 
   /**
-   * @param name of the function to describe
+   * @param name
+   *          of the function to describe
    */
   public descFunctionDesc(Path resFile, String name, boolean isExtended) {
     this.isExtended = isExtended;
@@ -74,13 +75,14 @@
   /**
    * @return the name
    */
-  @explain(displayName="name")
+  @explain(displayName = "name")
   public String getName() {
     return name;
   }
 
   /**
-   * @param name is the function name
+   * @param name
+   *          is the function name
    */
   public void setName(String name) {
     this.name = name;
@@ -93,12 +95,14 @@
     return resFile;
   }
 
-  @explain(displayName="result file", normalExplain=false)
+  @explain(displayName = "result file", normalExplain = false)
   public String getResFileString() {
     return getResFile().getName();
   }
+
   /**
-   * @param resFile the resFile to set
+   * @param resFile
+   *          the resFile to set
    */
   public void setResFile(Path resFile) {
     this.resFile = resFile;

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/descTableDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/descTableDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/descTableDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/descTableDesc.java Thu Jan 21 10:37:58 2010
@@ -23,15 +23,14 @@
 
 import org.apache.hadoop.fs.Path;
 
-@explain(displayName="Describe Table")
-public class descTableDesc extends ddlDesc implements Serializable 
-{
+@explain(displayName = "Describe Table")
+public class descTableDesc extends ddlDesc implements Serializable {
   private static final long serialVersionUID = 1L;
-  
+
   String tableName;
   HashMap<String, String> partSpec;
-  Path              resFile;
-  boolean           isExt;
+  Path resFile;
+  boolean isExt;
   /**
    * table name for the result of describe table
    */
@@ -48,14 +47,15 @@
   public String getSchema() {
     return schema;
   }
- 
+
   /**
    * @param isExt
    * @param partSpec
    * @param resFile
    * @param tableName
    */
-  public descTableDesc(Path resFile, String tableName, HashMap<String, String> partSpec, boolean isExt) {
+  public descTableDesc(Path resFile, String tableName,
+      HashMap<String, String> partSpec, boolean isExt) {
     this.isExt = isExt;
     this.partSpec = partSpec;
     this.resFile = resFile;
@@ -70,7 +70,8 @@
   }
 
   /**
-   * @param isExt the isExt to set
+   * @param isExt
+   *          the isExt to set
    */
   public void setExt(boolean isExt) {
     this.isExt = isExt;
@@ -79,13 +80,14 @@
   /**
    * @return the tableName
    */
-  @explain(displayName="table")
+  @explain(displayName = "table")
   public String getTableName() {
     return tableName;
   }
 
   /**
-   * @param tableName the tableName to set
+   * @param tableName
+   *          the tableName to set
    */
   public void setTableName(String tableName) {
     this.tableName = tableName;
@@ -94,13 +96,14 @@
   /**
    * @return the partSpec
    */
-  @explain(displayName="partition")
+  @explain(displayName = "partition")
   public HashMap<String, String> getPartSpec() {
     return partSpec;
   }
 
   /**
-   * @param partSpec the partSpec to set
+   * @param partSpec
+   *          the partSpec to set
    */
   public void setPartSpecs(HashMap<String, String> partSpec) {
     this.partSpec = partSpec;
@@ -113,13 +116,14 @@
     return resFile;
   }
 
-  @explain(displayName="result file", normalExplain=false)
+  @explain(displayName = "result file", normalExplain = false)
   public String getResFileString() {
     return getResFile().getName();
   }
-  
+
   /**
-   * @param resFile the resFile to set
+   * @param resFile
+   *          the resFile to set
    */
   public void setResFile(Path resFile) {
     this.resFile = resFile;

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/dropFunctionDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/dropFunctionDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/dropFunctionDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/dropFunctionDesc.java Thu Jan 21 10:37:58 2010
@@ -20,17 +20,17 @@
 
 import java.io.Serializable;
 
-@explain(displayName="Drop Function")
+@explain(displayName = "Drop Function")
 public class dropFunctionDesc implements Serializable {
   private static final long serialVersionUID = 1L;
-  
+
   private String functionName;
-  
+
   public dropFunctionDesc(String functionName) {
     this.functionName = functionName;
   }
 
-  @explain(displayName="name")
+  @explain(displayName = "name")
   public String getFunctionName() {
     return functionName;
   }
@@ -38,5 +38,5 @@
   public void setFunctionName(String functionName) {
     this.functionName = functionName;
   }
-  
+
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/dropTableDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/dropTableDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/dropTableDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/dropTableDesc.java Thu Jan 21 10:37:58 2010
@@ -19,16 +19,14 @@
 package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-@explain(displayName="Drop Table")
-public class dropTableDesc extends ddlDesc implements Serializable 
-{
+@explain(displayName = "Drop Table")
+public class dropTableDesc extends ddlDesc implements Serializable {
   private static final long serialVersionUID = 1L;
-  
-  String            tableName;
+
+  String tableName;
   List<Map<String, String>> partSpecs;
   boolean expectView;
 
@@ -37,26 +35,27 @@
    */
   public dropTableDesc(String tableName, boolean expectView) {
     this.tableName = tableName;
-    this.partSpecs = null;
+    partSpecs = null;
     this.expectView = expectView;
   }
 
   public dropTableDesc(String tableName, List<Map<String, String>> partSpecs) {
     this.tableName = tableName;
     this.partSpecs = partSpecs;
-    this.expectView = false;
+    expectView = false;
   }
 
   /**
    * @return the tableName
    */
-  @explain(displayName="table")
+  @explain(displayName = "table")
   public String getTableName() {
     return tableName;
   }
 
   /**
-   * @param tableName the tableName to set
+   * @param tableName
+   *          the tableName to set
    */
   public void setTableName(String tableName) {
     this.tableName = tableName;
@@ -70,7 +69,8 @@
   }
 
   /**
-   * @param partSpecs the partSpecs to set
+   * @param partSpecs
+   *          the partSpecs to set
    */
   public void setPartSpecs(List<Map<String, String>> partSpecs) {
     this.partSpecs = partSpecs;
@@ -84,7 +84,8 @@
   }
 
   /**
-   * @param expectView set whether to expect a view being dropped
+   * @param expectView
+   *          set whether to expect a view being dropped
    */
   public void setExpectView(boolean expectView) {
     this.expectView = expectView;

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explain.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explain.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explain.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explain.java Thu Jan 21 10:37:58 2010
@@ -24,5 +24,6 @@
 @Retention(RetentionPolicy.RUNTIME)
 public @interface explain {
   String displayName() default "";
+
   boolean normalExplain() default true;
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explainWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explainWork.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explainWork.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explainWork.java Thu Jan 21 10:37:58 2010
@@ -31,49 +31,48 @@
   private List<Task<? extends Serializable>> rootTasks;
   private String astStringTree;
   boolean extended;
-  
-  public explainWork() { }
-  
-  public explainWork(Path resFile, 
-                     List<Task<? extends Serializable>> rootTasks,
-                     String astStringTree,
-                     boolean extended) {
+
+  public explainWork() {
+  }
+
+  public explainWork(Path resFile,
+      List<Task<? extends Serializable>> rootTasks, String astStringTree,
+      boolean extended) {
     this.resFile = resFile;
     this.rootTasks = rootTasks;
     this.astStringTree = astStringTree;
     this.extended = extended;
   }
-  
+
   public Path getResFile() {
     return resFile;
   }
-  
+
   public void setResFile(Path resFile) {
     this.resFile = resFile;
   }
-  
+
   public List<Task<? extends Serializable>> getRootTasks() {
     return rootTasks;
   }
-  
+
   public void setRootTasks(List<Task<? extends Serializable>> rootTasks) {
     this.rootTasks = rootTasks;
   }
-  
+
   public String getAstStringTree() {
     return astStringTree;
   }
-  
+
   public void setAstStringTree(String astStringTree) {
     this.astStringTree = astStringTree;
   }
-  
+
   public boolean getExtended() {
     return extended;
   }
-  
+
   public void setExtended(boolean extended) {
     this.extended = extended;
   }
 }
-

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explosionDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explosionDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explosionDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/explosionDesc.java Thu Jan 21 10:37:58 2010
@@ -16,33 +16,37 @@
  * limitations under the License.
  */
 
-
 package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
-@explain(displayName="Explosion")
+@explain(displayName = "Explosion")
 public class explosionDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   private String fieldName;
   private int position;
-  public explosionDesc() { }
-  public explosionDesc(
-    final String fieldName,
-    final int position) {
+
+  public explosionDesc() {
+  }
+
+  public explosionDesc(final String fieldName, final int position) {
     this.fieldName = fieldName;
     this.position = position;
   }
+
   public String getFieldName() {
-    return this.fieldName;
+    return fieldName;
   }
+
   public void setFieldName(final String fieldName) {
-    this.fieldName=fieldName;
+    this.fieldName = fieldName;
   }
+
   public int getPosition() {
-    return this.position;
+    return position;
   }
+
   public void setPosition(final int position) {
-    this.position=position;
+    this.position = position;
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeColumnDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeColumnDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeColumnDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeColumnDesc.java Thu Jan 21 10:37:58 2010
@@ -37,77 +37,90 @@
    * The alias of the table.
    */
   private String tabAlias;
-  
+
   /**
    * Is the column a partitioned column.
    */
   private boolean isPartitionCol;
-  
-  public exprNodeColumnDesc() {}
-  public exprNodeColumnDesc(TypeInfo typeInfo, String column,
-                            String tabAlias, boolean isPartitionCol) {
+
+  public exprNodeColumnDesc() {
+  }
+
+  public exprNodeColumnDesc(TypeInfo typeInfo, String column, String tabAlias,
+      boolean isPartitionCol) {
     super(typeInfo);
     this.column = column;
     this.tabAlias = tabAlias;
     this.isPartitionCol = isPartitionCol;
   }
+
   public exprNodeColumnDesc(Class<?> c, String column, String tabAlias,
-                            boolean isPartitionCol) {
+      boolean isPartitionCol) {
     super(TypeInfoFactory.getPrimitiveTypeInfoFromJavaPrimitive(c));
     this.column = column;
     this.tabAlias = tabAlias;
     this.isPartitionCol = isPartitionCol;
   }
+
   public String getColumn() {
-    return this.column;
+    return column;
   }
+
   public void setColumn(String column) {
     this.column = column;
   }
 
   public String getTabAlias() {
-    return this.tabAlias;
+    return tabAlias;
   }
+
   public void setTabAlias(String tabAlias) {
     this.tabAlias = tabAlias;
   }
 
   public boolean getIsParititonCol() {
-    return this.isPartitionCol;
+    return isPartitionCol;
   }
+
   public void setIsPartitionCol(boolean isPartitionCol) {
     this.isPartitionCol = isPartitionCol;
   }
 
+  @Override
   public String toString() {
     return "Column[" + column + "]";
   }
-  
-  @explain(displayName="expr")
+
+  @explain(displayName = "expr")
   @Override
   public String getExprString() {
     return getColumn();
   }
 
+  @Override
   public List<String> getCols() {
-  	List<String> lst = new ArrayList<String>();
-  	lst.add(column);
-  	return lst;
+    List<String> lst = new ArrayList<String>();
+    lst.add(column);
+    return lst;
   }
+
   @Override
   public exprNodeDesc clone() {
-    return new exprNodeColumnDesc(this.typeInfo, this.column, 
-                                  this.tabAlias, this.isPartitionCol);
+    return new exprNodeColumnDesc(typeInfo, column, tabAlias, isPartitionCol);
   }
+
   @Override
   public boolean isSame(Object o) {
-    if (!(o instanceof exprNodeColumnDesc))
+    if (!(o instanceof exprNodeColumnDesc)) {
       return false;
-    exprNodeColumnDesc dest = (exprNodeColumnDesc)o;
-    if (!column.equals(dest.getColumn()))
+    }
+    exprNodeColumnDesc dest = (exprNodeColumnDesc) o;
+    if (!column.equals(dest.getColumn())) {
       return false;
-    if (!typeInfo.equals(dest.getTypeInfo()))
+    }
+    if (!typeInfo.equals(dest.getTypeInfo())) {
       return false;
-    return true; 
+    }
+    return true;
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeConstantDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeConstantDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeConstantDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeConstantDesc.java Thu Jan 21 10:37:58 2010
@@ -30,30 +30,34 @@
 public class exprNodeConstantDesc extends exprNodeDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   private Object value;
-  
-  public exprNodeConstantDesc() {}
+
+  public exprNodeConstantDesc() {
+  }
+
   public exprNodeConstantDesc(TypeInfo typeInfo, Object value) {
     super(typeInfo);
     this.value = value;
   }
+
   public exprNodeConstantDesc(Object value) {
-    this(TypeInfoFactory.getPrimitiveTypeInfoFromJavaPrimitive(value.getClass()),
-        value);
+    this(TypeInfoFactory
+        .getPrimitiveTypeInfoFromJavaPrimitive(value.getClass()), value);
   }
 
   public void setValue(Object value) {
     this.value = value;
   }
-  
+
   public Object getValue() {
-    return this.value;
+    return value;
   }
 
+  @Override
   public String toString() {
     return "Const " + typeInfo.toString() + " " + value;
   }
-  
-  @explain(displayName="expr")
+
+  @explain(displayName = "expr")
   @Override
   public String getExprString() {
     if (value == null) {
@@ -62,25 +66,29 @@
 
     if (typeInfo.getTypeName().equals(Constants.STRING_TYPE_NAME)) {
       return "'" + value.toString() + "'";
-    }
-    else {
+    } else {
       return value.toString();
     }
   }
+
   @Override
   public exprNodeDesc clone() {
-    return new exprNodeConstantDesc(this.typeInfo, this.value);
+    return new exprNodeConstantDesc(typeInfo, value);
   }
+
   @Override
   public boolean isSame(Object o) {
-    if (!(o instanceof exprNodeConstantDesc))
+    if (!(o instanceof exprNodeConstantDesc)) {
       return false;
-    exprNodeConstantDesc dest = (exprNodeConstantDesc)o;
-    if (!typeInfo.equals(dest.getTypeInfo()))
+    }
+    exprNodeConstantDesc dest = (exprNodeConstantDesc) o;
+    if (!typeInfo.equals(dest.getTypeInfo())) {
       return false;
-    if (!value.equals(dest.getValue()))
+    }
+    if (!value.equals(dest.getValue())) {
       return false;
-        
-    return true; 
+    }
+
+    return true;
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeDesc.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeDesc.java Thu Jan 21 10:37:58 2010
@@ -24,38 +24,43 @@
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
-public abstract class exprNodeDesc implements Serializable, Node {  
+public abstract class exprNodeDesc implements Serializable, Node {
   private static final long serialVersionUID = 1L;
   TypeInfo typeInfo;
-  
-  public exprNodeDesc() {}
+
+  public exprNodeDesc() {
+  }
+
   public exprNodeDesc(TypeInfo typeInfo) {
     this.typeInfo = typeInfo;
     if (typeInfo == null) {
       throw new RuntimeException("typeInfo cannot be null!");
     }
   }
-  
+
+  @Override
   public abstract exprNodeDesc clone();
 
   // Cant use equals because the walker depends on them being object equal
-  // The default graph walker processes a node after its kids have been processed. That comparison needs
+  // The default graph walker processes a node after its kids have been
+  // processed. That comparison needs
   // object equality - isSame means that the objects are semantically equal.
   public abstract boolean isSame(Object o);
-  
+
   public TypeInfo getTypeInfo() {
-    return this.typeInfo;
+    return typeInfo;
   }
+
   public void setTypeInfo(TypeInfo typeInfo) {
     this.typeInfo = typeInfo;
   }
 
   public String getExprString() {
-    assert(false);
+    assert (false);
     return null;
   }
-  
-  @explain(displayName="type")
+
+  @explain(displayName = "type")
   public String getTypeString() {
     return typeInfo.getTypeName();
   }
@@ -63,12 +68,12 @@
   public List<String> getCols() {
     return null;
   }
-  
+
   @Override
   public List<exprNodeDesc> getChildren() {
     return null;
   }
-  
+
   @Override
   public String getName() {
     return this.getClass().getName();