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

svn commit: r1243013 [9/9] - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ conf/ contrib/src/test/results/clientnegative/ contrib/src/test/results/clientpositive/ metastore/if/ metastore/scripts/upgrade/derby/ metastore/scripts/upgrade/m...

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java Sat Feb 11 07:49:28 2012
@@ -239,6 +239,13 @@ public abstract class BaseSemanticAnalyz
   public abstract void analyzeInternal(ASTNode ast) throws SemanticException;
 
   public void analyze(ASTNode ast, Context ctx) throws SemanticException {
+    boolean useDefaultRegion =
+      HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_USE_INPUT_PRIMARY_REGION);
+
+    if (!useDefaultRegion) {
+      throw new SemanticException(ErrorMsg.USE_DEFAULT_REGION.getMsg());
+    }
+
     this.ctx = ctx;
     analyzeInternal(ast);
   }
@@ -777,7 +784,7 @@ public abstract class BaseSemanticAnalyz
     }
     return partSpec;
   }
-  
+
   /**
    * Checks if given specification is proper specification for prefix of
    * partition cols, for table partitioned by ds, hr, min valid ones are
@@ -805,7 +812,7 @@ public abstract class BaseSemanticAnalyz
     if (spec == null) {
       throw new HiveException("partition spec is not specified");
     }
-    
+
     Iterator<String> itrPsKeys = spec.keySet().iterator();
     for (FieldSchema fs: partCols) {
       if(!itrPsKeys.hasNext()) {
@@ -816,14 +823,14 @@ public abstract class BaseSemanticAnalyz
         ErrorPartSpec(spec, partCols);
       }
     }
-    
+
     if(itrPsKeys.hasNext()) {
       ErrorPartSpec(spec, partCols);
     }
 
     return true;
   }
-  
+
   private static void ErrorPartSpec(Map<String, String> partSpec,
       List<FieldSchema> parts) throws SemanticException {
     StringBuilder sb =

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java Sat Feb 11 07:49:28 2012
@@ -106,6 +106,7 @@ public enum ErrorMsg {
       + "denaminator in sample clause for table"),
   NEED_PARTITION_ERROR("Need to specify partition columns because the destination "
       + "table is partitioned"),
+  USE_DEFAULT_REGION("Only default region can be used"),
   CTAS_CTLT_COEXISTENCE("Create table command does not allow LIKE and AS-SELECT in "
       + "the same command"),
   LINES_TERMINATED_BY_NON_NEWLINE("LINES TERMINATED BY only supports newline '\\n' right now"),

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java Sat Feb 11 07:49:28 2012
@@ -111,7 +111,8 @@ public class ImportSemanticAnalyzer exte
             table.getSd().getSerdeInfo().getSerializationLib(),
             null, // storagehandler passed as table params
             table.getSd().getSerdeInfo().getParameters(),
-            table.getParameters(), false);
+            table.getParameters(), false,
+            table.getSd().getPrimaryRegionName());
 
 
         List<FieldSchema> partCols = tblDesc.getPartCols();

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Sat Feb 11 07:49:28 2012
@@ -7944,6 +7944,8 @@ public class SemanticAnalyzer extends Ba
     RowFormatParams rowFormatParams = new RowFormatParams();
     StorageFormat storageFormat = new StorageFormat();
     AnalyzeCreateCommonVars shared = new AnalyzeCreateCommonVars();
+    String defaultRegionName =
+      HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DEFAULT_REGION_NAME);
 
     LOG.info("Creating table " + tableName + " position="
         + ast.getCharPositionInLine());
@@ -8082,7 +8084,9 @@ public class SemanticAnalyzer extends Ba
           bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim, rowFormatParams.fieldEscape,
           rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim, comment,
           storageFormat.inputFormat, storageFormat.outputFormat, location, shared.serde,
-          storageFormat.storageHandler, shared.serdeProps, tblProps, ifNotExists);
+          storageFormat.storageHandler, shared.serdeProps,
+          tblProps, ifNotExists,
+          defaultRegionName);
 
       validateCreateTable(crtTblDesc);
       // outputs is empty, which means this create table happens in the current
@@ -8124,7 +8128,8 @@ public class SemanticAnalyzer extends Ba
           bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim, rowFormatParams.fieldEscape,
           rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim, comment, storageFormat.inputFormat,
           storageFormat.outputFormat, location, shared.serde, storageFormat.storageHandler, shared.serdeProps,
-          tblProps, ifNotExists);
+          tblProps, ifNotExists,
+          defaultRegionName);
       qb.setTableDesc(crtTblDesc);
 
       SessionState.get().setCommandType(HiveOperation.CREATETABLE_AS_SELECT);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java Sat Feb 11 07:49:28 2012
@@ -56,6 +56,7 @@ public class CreateTableDesc extends DDL
   Map<String, String> serdeProps;
   Map<String, String> tblProps;
   boolean ifNotExists;
+  String primaryRegionName;
 
   public CreateTableDesc() {
   }
@@ -69,13 +70,14 @@ public class CreateTableDesc extends DDL
       String storageHandler,
       Map<String, String> serdeProps,
       Map<String, String> tblProps,
-      boolean ifNotExists) {
+      boolean ifNotExists,
+      String primaryRegionName) {
 
     this(tableName, isExternal, cols, partCols,
         bucketCols, sortCols, numBuckets, fieldDelim, fieldEscape,
         collItemDelim, mapKeyDelim, lineDelim, comment, inputFormat,
         outputFormat, location, serName, storageHandler, serdeProps,
-        tblProps, ifNotExists);
+        tblProps, ifNotExists, primaryRegionName);
 
     this.databaseName = databaseName;
   }
@@ -89,7 +91,8 @@ public class CreateTableDesc extends DDL
       String storageHandler,
       Map<String, String> serdeProps,
       Map<String, String> tblProps,
-      boolean ifNotExists) {
+      boolean ifNotExists,
+      String primaryRegionName) {
     this.tableName = tableName;
     this.isExternal = isExternal;
     this.bucketCols = new ArrayList<String>(bucketCols);
@@ -111,6 +114,7 @@ public class CreateTableDesc extends DDL
     this.serdeProps = serdeProps;
     this.tblProps = tblProps;
     this.ifNotExists = ifNotExists;
+    this.primaryRegionName = primaryRegionName;
   }
 
   @Explain(displayName = "columns")
@@ -342,4 +346,19 @@ public class CreateTableDesc extends DDL
     this.tblProps = tblProps;
   }
 
+  /**
+   * @return the primaryRegionName
+   */
+  @Explain(displayName = "primaryRegionName")
+  public String getPrimaryRegionName() {
+    return primaryRegionName;
+  }
+
+  /**
+   * @param primaryRegionName
+   *          the primaryRegionName to set
+   */
+  public void setPrimaryRegionName(String primaryRegionName) {
+    this.primaryRegionName = primaryRegionName;
+  }
 }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java Sat Feb 11 07:49:28 2012
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.RegionStorageDescriptor;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
@@ -135,6 +136,9 @@ public class TestHive extends TestCase {
       tbl.setSerdeParam(Constants.FIELD_DELIM, "1");
       tbl.setSerializationLib(LazySimpleSerDe.class.getName());
 
+      tbl.getTTable().getSd().setPrimaryRegionName(HiveConf.ConfVars.HIVE_DEFAULT_REGION_NAME.defaultVal);
+      tbl.getTTable().getSd().setSecondaryRegions(new ArrayList<RegionStorageDescriptor>());
+
       // create table
       try {
         hm.createTable(tbl);
@@ -209,6 +213,11 @@ public class TestHive extends TestCase {
       tbl.setSerdeParam(Constants.SERIALIZATION_CLASS, Complex.class.getName());
       tbl.setSerdeParam(Constants.SERIALIZATION_FORMAT, TBinaryProtocol.class
           .getName());
+
+      tbl.getTTable().getSd().setPrimaryRegionName(HiveConf.ConfVars.HIVE_DEFAULT_REGION_NAME.defaultVal);
+      tbl.getTTable().getSd().setSecondaryRegions(new ArrayList<RegionStorageDescriptor>());
+
+
       try {
         hm.createTable(tbl);
       } catch (HiveException e) {

Modified: hive/trunk/ql/src/test/results/clientpositive/create_union_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/create_union_table.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/create_union_table.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/create_union_table.q.out Sat Feb 11 07:49:28 2012
@@ -19,6 +19,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: abc
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/ctas.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/ctas.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/ctas.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/ctas.q.out Sat Feb 11 07:49:28 2012
@@ -104,6 +104,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: nzhang_CTAS1
           isExternal: false
 
@@ -263,6 +264,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: nzhang_ctas2
           isExternal: false
 
@@ -422,6 +424,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat
+          primaryRegionName: 
           serde name: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
           name: nzhang_ctas3
           isExternal: false
@@ -650,6 +653,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: nzhang_ctas4
           isExternal: false
 
@@ -893,6 +897,7 @@ STAGE PLANS:
 
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: nzhang_ctas5
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/fileformat_sequencefile.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/fileformat_sequencefile.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/fileformat_sequencefile.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/fileformat_sequencefile.q.out Sat Feb 11 07:49:28 2012
@@ -23,6 +23,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
           # buckets: -1
           output format: org.apache.hadoop.mapred.SequenceFileOutputFormat
+          primaryRegionName: 
           name: dest1
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/fileformat_text.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/fileformat_text.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/fileformat_text.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/fileformat_text.q.out Sat Feb 11 07:49:28 2012
@@ -23,6 +23,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: dest1
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/input15.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/input15.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/input15.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/input15.q.out Sat Feb 11 07:49:28 2012
@@ -20,6 +20,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: TEST15
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/inputddl1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/inputddl1.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/inputddl1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/inputddl1.q.out Sat Feb 11 07:49:28 2012
@@ -19,6 +19,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: INPUTDDL1
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/inputddl2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/inputddl2.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/inputddl2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/inputddl2.q.out Sat Feb 11 07:49:28 2012
@@ -20,6 +20,7 @@ STAGE PLANS:
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
           partition columns: ds string, country string
+          primaryRegionName: 
           name: INPUTDDL2
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/inputddl3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/inputddl3.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/inputddl3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/inputddl3.q.out Sat Feb 11 07:49:28 2012
@@ -20,6 +20,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: INPUTDDL3
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/merge3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/merge3.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/merge3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/merge3.q.out Sat Feb 11 07:49:28 2012
@@ -164,6 +164,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: merge_src2
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/rcfile_createas1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/rcfile_createas1.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/rcfile_createas1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/rcfile_createas1.q.out Sat Feb 11 07:49:28 2012
@@ -114,6 +114,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat
+          primaryRegionName: 
           serde name: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
           name: rcfile_createas1b
           isExternal: false

Modified: hive/trunk/ql/src/test/results/clientpositive/smb_mapjoin9.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/smb_mapjoin9.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/smb_mapjoin9.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/smb_mapjoin9.q.out Sat Feb 11 07:49:28 2012
@@ -126,6 +126,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: smb_mapjoin9_results
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/symlink_text_input_format.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/symlink_text_input_format.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/symlink_text_input_format.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/symlink_text_input_format.q.out Sat Feb 11 07:49:28 2012
@@ -19,6 +19,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: symlink_text_input_format
           isExternal: false
 

Modified: hive/trunk/ql/src/test/results/clientpositive/union25.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/union25.q.out?rev=1243013&r1=1243012&r2=1243013&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/union25.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/union25.q.out Sat Feb 11 07:49:28 2012
@@ -286,6 +286,7 @@ STAGE PLANS:
           input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
           output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          primaryRegionName: 
           name: tmp_unionall
           isExternal: false