You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2014/09/27 00:18:23 UTC

svn commit: r1627903 - in /hive/branches/branch-0.14/ql/src: java/org/apache/hadoop/hive/ql/optimizer/ java/org/apache/hadoop/hive/ql/parse/ java/org/apache/hadoop/hive/ql/plan/ test/results/compiler/plan/

Author: gunther
Date: Fri Sep 26 22:18:22 2014
New Revision: 1627903

URL: http://svn.apache.org/r1627903
Log:
HIVE-8072: TesParse_union is failing on trunk (Navis via Gunther Hagleitner)

Modified:
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input1.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input2.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input3.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input6.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input7.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input9.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample2.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample3.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample4.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample5.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample6.q.xml
    hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample7.q.xml

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java Fri Sep 26 22:18:22 2014
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.optimizer;
 
+import com.google.common.collect.Interner;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
@@ -39,8 +40,6 @@ import org.apache.hadoop.hive.ql.exec.No
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.OperatorUtils;
-import org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator;
-import org.apache.hadoop.hive.ql.exec.RCFileMergeOperator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
@@ -99,7 +98,6 @@ import org.apache.hadoop.hive.ql.plan.Ta
 import org.apache.hadoop.hive.ql.plan.TezWork;
 import org.apache.hadoop.hive.ql.stats.StatsFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.InputFormat;
 
 import java.io.Serializable;
@@ -873,6 +871,30 @@ public final class GenMapRedUtils {
     }
   }
 
+  public static void internTableDesc(Task<?> task, Interner<TableDesc> interner) {
+
+    if (task instanceof ConditionalTask) {
+      for (Task tsk : ((ConditionalTask) task).getListTasks()) {
+        internTableDesc(tsk, interner);
+      }
+    } else if (task instanceof ExecDriver) {
+      MapredWork work = (MapredWork) task.getWork();
+      work.getMapWork().internTable(interner);
+    } else if (task != null && (task.getWork() instanceof TezWork)) {
+      TezWork work = (TezWork)task.getWork();
+      for (BaseWork w : work.getAllWorkUnsorted()) {
+        if (w instanceof MapWork) {
+          ((MapWork)w).internTable(interner);
+        }
+      }
+    }
+    if (task.getNumChild() > 0) {
+      for (Task childTask : task.getChildTasks()) {
+        internTableDesc(childTask, interner);
+      }
+    }
+  }
+
   /**
    * create a new plan and return.
    *

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java Fri Sep 26 22:18:22 2014
@@ -26,6 +26,8 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
+import com.google.common.collect.Interner;
+import com.google.common.collect.Interners;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
@@ -278,6 +280,11 @@ public abstract class TaskCompiler {
         tsk.setRetryCmdWhenFail(true);
       }
     }
+
+    Interner<TableDesc> interner = Interners.newStrongInterner();
+    for (Task<? extends Serializable> rootTask : rootTasks) {
+      GenMapRedUtils.internTableDesc(rootTask, interner);
+    }
   }
 
 

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java Fri Sep 26 22:18:22 2014
@@ -30,6 +30,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import com.google.common.collect.Interner;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
@@ -198,6 +199,22 @@ public class MapWork extends BaseWork {
     }
   }
 
+  public void internTable(Interner<TableDesc> interner) {
+    if (aliasToPartnInfo != null) {
+      for (PartitionDesc part : aliasToPartnInfo.values()) {
+        if (part == null) {
+          continue;
+        }
+        part.intern(interner);
+      }
+    }
+    if (pathToPartitionInfo != null) {
+      for (PartitionDesc part : pathToPartitionInfo.values()) {
+        part.intern(interner);
+      }
+    }
+  }
+
   /**
    * @return the aliasToPartnInfo
    */

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java Fri Sep 26 22:18:22 2014
@@ -48,12 +48,10 @@ import org.apache.hadoop.util.Reflection
 public class PartitionDesc implements Serializable, Cloneable {
 
   static {
-    TABLE_INTERNER = Interners.newWeakInterner();
     STRING_INTERNER = Interners.newWeakInterner();
     CLASS_INTERNER = Interners.newWeakInterner();
   }
 
-  private static final Interner<TableDesc> TABLE_INTERNER;
   private static final Interner<String> STRING_INTERNER;
   private static final Interner<Class<?>> CLASS_INTERNER;
 
@@ -73,12 +71,12 @@ public class PartitionDesc implements Se
   }
 
   public PartitionDesc(final TableDesc table, final LinkedHashMap<String, String> partSpec) {
-    setTableDesc(table);
+    this.tableDesc = table;
     this.partSpec = partSpec;
   }
 
   public PartitionDesc(final Partition part) throws HiveException {
-    setTableDesc(Utilities.getTableDesc(part.getTable()));
+    this.tableDesc = Utilities.getTableDesc(part.getTable());
     setProperties(part.getMetadataFromPartitionSchema());
     partSpec = part.getSpec();
     setInputFileFormatClass(part.getInputFormatClass());
@@ -86,7 +84,7 @@ public class PartitionDesc implements Se
   }
 
   public PartitionDesc(final Partition part,final TableDesc tblDesc) throws HiveException {
-    setTableDesc(tblDesc);
+    this.tableDesc = tblDesc;
     setProperties(part.getSchemaFromTableSchema(tblDesc.getProperties())); // each partition maintains a large properties
     partSpec = part.getSpec();
     setOutputFileFormatClass(part.getInputFormatClass());
@@ -99,7 +97,7 @@ public class PartitionDesc implements Se
   }
 
   public void setTableDesc(TableDesc tableDesc) {
-    this.tableDesc = TABLE_INTERNER.intern(tableDesc);
+    this.tableDesc = tableDesc;
   }
 
   @Explain(displayName = "partition values")
@@ -266,4 +264,8 @@ public class PartitionDesc implements Se
       baseFileName = path;
     }
   }
+
+  public void intern(Interner<TableDesc> interner) {
+    this.tableDesc = interner.intern(tableDesc);
+  }
 }

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input1.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input1.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input1.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input1.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input2.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input2.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input2.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input2.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input3.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input3.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input3.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input3.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 
@@ -881,7 +822,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc1" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -979,66 +920,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc1" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest2</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest2 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc1"/> 
                   </void> 
                  </object> 
                 </void> 
@@ -1440,7 +1322,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc2" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -1547,70 +1429,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc2" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest3</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest3 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>partition_columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc2"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input6.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input6.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input6.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input6.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input7.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input7.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input7.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input7.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input9.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input9.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input9.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/input9.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample2.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample2.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample2.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample2.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample3.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample3.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample3.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample3.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample4.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample4.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample4.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample4.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample5.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample5.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample5.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample5.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample6.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample6.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample6.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample6.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void> 

Modified: hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample7.q.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample7.q.xml?rev=1627903&r1=1627902&r2=1627903&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample7.q.xml (original)
+++ hive/branches/branch-0.14/ql/src/test/results/compiler/plan/sample7.q.xml Fri Sep 26 22:18:22 2014
@@ -334,7 +334,7 @@
                          </object> 
                         </void> 
                         <void property="tableDesc"> 
-                         <object class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
+                         <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
                           <void property="inputFileFormatClass"> 
                            <class>org.apache.hadoop.mapred.TextInputFormat</class> 
                           </void> 
@@ -432,66 +432,7 @@
                    <boolean>true</boolean> 
                   </void> 
                   <void property="table"> 
-                   <object id="TableDesc0" class="org.apache.hadoop.hive.ql.plan.TableDesc"> 
-                    <void property="inputFileFormatClass"> 
-                     <class>org.apache.hadoop.mapred.TextInputFormat</class> 
-                    </void> 
-                    <void property="outputFileFormatClass"> 
-                     <class>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</class> 
-                    </void> 
-                    <void property="properties"> 
-                     <object class="java.util.Properties"> 
-                      <void method="put"> 
-                       <string>name</string> 
-                       <string>default.dest1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.types</string> 
-                       <string>string:string</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.ddl</string> 
-                       <string>struct dest1 { string key, string value}</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.format</string> 
-                       <string>1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns</string> 
-                       <string>key,value</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>columns.comments</string> 
-                       <string>default<char code="#0"/>default</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>bucket_count</string> 
-                       <string>-1</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>serialization.lib</string> 
-                       <string>org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.inputformat</string> 
-                       <string>org.apache.hadoop.mapred.TextInputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>file.outputformat</string> 
-                       <string>org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat</string> 
-                      </void> 
-                      <void method="put"> 
-                       <string>location</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                      <void method="put"> 
-                       <string>transient_lastDdlTime</string> 
-                       #### A masked pattern was here #### 
-                      </void> 
-                     </object> 
-                    </void> 
-                   </object> 
+                   <object idref="TableDesc0"/> 
                   </void> 
                  </object> 
                 </void>