You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2008/12/24 00:40:45 UTC

svn commit: r729176 - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/java/org/apache/hadoop/hive/ql/plan/ ql/src/test/queries/clientnegative/ ql/src/test/queries/clientpositive/...

Author: zshao
Date: Tue Dec 23 15:40:45 2008
New Revision: 729176

URL: http://svn.apache.org/viewvc?rev=729176&view=rev
Log:
HIVE-177. Allow specifying custom input and output format in CREATE TABLE. (David Phillips via zshao)

Added:
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_bad_class.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_input.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_output.q
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_sequencefile.q
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_text.q
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_void.q
    hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_bad_class.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_output.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_sequencefile.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_text.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_void.q.out
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input15.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl3.q.out

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Tue Dec 23 15:40:45 2008
@@ -29,6 +29,9 @@
 
   IMPROVEMENTS
 
+    HIVE-177. Allow specifying custom input and output format in CREATE TABLE.
+    (David Phillips via zshao)
+
     HIVE-166. Create eclipse project template. (Ashish Thusoo via zshao)
 
     HIVE-186. Refactor code to use a single graph, nodeprocessor, dispatcher

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Tue Dec 23 15:40:45 2008
@@ -41,7 +41,6 @@
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
@@ -58,9 +57,6 @@
 import org.apache.hadoop.hive.serde.thrift.columnsetSerDe;
 import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
 import org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
-import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.util.StringUtils;
 
 import com.facebook.thrift.TException;
@@ -551,13 +547,8 @@
     if (crtTbl.getLocation() != null)
       tblStorDesc.setLocation(crtTbl.getLocation());
 
-    if (crtTbl.isSequenceFile()) {
-      tbl.setInputFormatClass(SequenceFileInputFormat.class);
-      tbl.setOutputFormatClass(SequenceFileOutputFormat.class);
-    } else {
-      tbl.setOutputFormatClass(IgnoreKeyTextOutputFormat.class);
-      tbl.setInputFormatClass(TextInputFormat.class);
-    }
+    tbl.setInputFormatClass(crtTbl.getInputFormat());
+    tbl.setOutputFormatClass(crtTbl.getOutputFormat());
 
     if (crtTbl.isExternal())
       tbl.setProperty("EXTERNAL", "TRUE");

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Tue Dec 23 15:40:45 2008
@@ -35,6 +35,7 @@
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.alterTableDesc;
 import org.apache.hadoop.hive.ql.plan.createTableDesc;
@@ -45,6 +46,9 @@
 import org.apache.hadoop.hive.ql.plan.alterTableDesc.alterTableTypes;
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
 
 public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   private static final Log LOG = LogFactory.getLog("hive.ql.parse.DDLSemanticAnalyzer");
@@ -62,6 +66,10 @@
     TokenToTypeName.put(HiveParser.TOK_DATETIME, Constants.DATETIME_TYPE_NAME);
     TokenToTypeName.put(HiveParser.TOK_TIMESTAMP, Constants.TIMESTAMP_TYPE_NAME);
   }
+  private static final String TEXTFILE_INPUT = TextInputFormat.class.getName();
+  private static final String TEXTFILE_OUTPUT = IgnoreKeyTextOutputFormat.class.getName();
+  private static final String SEQUENCEFILE_INPUT = SequenceFileInputFormat.class.getName();
+  private static final String SEQUENCEFILE_OUTPUT = SequenceFileOutputFormat.class.getName();
 
   public static String getTypeName(int token) {
     return TokenToTypeName.get(token);
@@ -127,12 +135,17 @@
     String            mapKeyDelim   = null;
     String            lineDelim     = null;
     String            comment       = null;
-    boolean           isSequenceFile  = 
-      "SequenceFile".equalsIgnoreCase(conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT));
+    String            inputFormat   = TEXTFILE_INPUT;
+    String            outputFormat  = TEXTFILE_OUTPUT;
     String            location      = null;
     String            serde         = null;
     Map<String, String> mapProp     = null;
 
+    if ("SequenceFile".equalsIgnoreCase(conf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT))) {
+      inputFormat = SEQUENCEFILE_INPUT;
+      outputFormat = SEQUENCEFILE_OUTPUT;
+    }
+
     LOG.info("Creating table" + tableName);    
     int numCh = ast.getChildCount();
     for (int num = 1; num < numCh; num++)
@@ -193,10 +206,16 @@
           }
           break;
         case HiveParser.TOK_TBLSEQUENCEFILE:
-          isSequenceFile = true;
+          inputFormat = SEQUENCEFILE_INPUT;
+          outputFormat = SEQUENCEFILE_OUTPUT;
           break;
         case HiveParser.TOK_TBLTEXTFILE:
-          isSequenceFile = false;
+          inputFormat = TEXTFILE_INPUT;
+          outputFormat = TEXTFILE_OUTPUT;
+          break;
+        case HiveParser.TOK_TABLEFILEFORMAT:
+          inputFormat = unescapeSQLString(child.getChild(0).getText());
+          outputFormat = unescapeSQLString(child.getChild(1).getText());
           break;
         case HiveParser.TOK_TABLELOCATION:
           location = unescapeSQLString(child.getChild(0).getText());
@@ -209,7 +228,7 @@
       new createTableDesc(tableName, isExt, cols, partCols, bucketCols, 
                           sortCols, numBuckets,
                           fieldDelim, collItemDelim, mapKeyDelim, lineDelim,
-                          comment, isSequenceFile, location, serde, mapProp);
+                          comment, inputFormat, outputFormat, location, serde, mapProp);
 
     validateCreateTable(crtTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(crtTblDesc), conf));

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java Tue Dec 23 15:40:45 2008
@@ -67,6 +67,8 @@
   CLUSTERBY_DISTRIBUTEBY_CONFLICT("Cannot have both Cluster By and Distribute By Clauses"),
   CLUSTERBY_SORTBY_CONFLICT("Cannot have both Cluster By and Sort By Clauses"),
   UNION_NOTIN_SUBQ("Top level Union is not supported currently; use a subquery for the union"),
+  INVALID_INPUT_FORMAT_TYPE("Input Format must implement InputFormat"),
+  INVALID_OUTPUT_FORMAT_TYPE("Output Format must implement OutputFormat"),
   NON_BUCKETED_TABLE("Sampling Expression Needed for Non-Bucketed Table");
 
   private String mesg;

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g Tue Dec 23 15:40:45 2008
@@ -104,6 +104,7 @@
 TOK_TABLEROWFORMATLINES;
 TOK_TBLSEQUENCEFILE;
 TOK_TBLTEXTFILE;
+TOK_TABLEFILEFORMAT;
 TOK_TABCOLNAME;
 TOK_TABLELOCATION;
 TOK_TABLESAMPLE;
@@ -305,6 +306,8 @@
     :
       KW_STORED KW_AS KW_SEQUENCEFILE  -> TOK_TBLSEQUENCEFILE
       | KW_STORED KW_AS KW_TEXTFILE  -> TOK_TBLTEXTFILE
+      | KW_STORED KW_AS KW_INPUTFORMAT inFmt=StringLiteral KW_OUTPUTFORMAT outFmt=StringLiteral
+      -> ^(TOK_TABLEFILEFORMAT $inFmt $outFmt)
     ;
 
 tableLocation
@@ -900,6 +903,8 @@
 KW_STORED: 'STORED';
 KW_SEQUENCEFILE: 'SEQUENCEFILE';
 KW_TEXTFILE: 'TEXTFILE';
+KW_INPUTFORMAT: 'INPUTFORMAT';
+KW_OUTPUTFORMAT: 'OUTPUTFORMAT';
 KW_LOCATION: 'LOCATION';
 KW_TABLESAMPLE: 'TABLESAMPLE';
 KW_BUCKET: 'BUCKET';

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Tue Dec 23 15:40:45 2008
@@ -58,6 +58,8 @@
 import org.apache.hadoop.hive.ql.exec.*;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.hive.ql.lib.Node;
 
@@ -596,6 +598,9 @@
           throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(qb.getParseInfo().getSrcForAlias(alias)));
         }
 
+        if (!InputFormat.class.isAssignableFrom(tab.getInputFormatClass()))
+          throw new SemanticException(ErrorMsg.INVALID_INPUT_FORMAT_TYPE.getMsg(qb.getParseInfo().getSrcForAlias(alias)));
+
         qb.getMetaData().setSrcForAlias(alias, tab);
       }
 
@@ -617,6 +622,9 @@
         case HiveParser.TOK_TAB: {
           tableSpec ts = new tableSpec(this.db, ast, true);
 
+          if (!OutputFormat.class.isAssignableFrom(ts.tableHandle.getOutputFormatClass()))
+            throw new SemanticException(ErrorMsg.INVALID_OUTPUT_FORMAT_TYPE.getMsg(ast));
+
           if(ts.partSpec == null) {
             // This is a table
             qb.getMetaData().setDestForAlias(name, ts.tableHandle);

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java Tue Dec 23 15:40:45 2008
@@ -42,7 +42,8 @@
   String              mapKeyDelim;
   String              lineDelim;
   String              comment;
-  boolean             isSequenceFile;
+  String              inputFormat;
+  String              outputFormat;
   String              location;
   String              serName;
   Map<String, String> mapProp;
@@ -53,7 +54,7 @@
                          int numBuckets, String fieldDelim, 
                          String collItemDelim,
                          String mapKeyDelim, String lineDelim, 
-                         String comment, boolean isSequenceFile, 
+                         String comment, String inputFormat, String outputFormat,
                          String location, String serName, Map<String, String> mapProp) {
     this.tableName       = tableName;
     this.isExternal      = isExternal;
@@ -63,7 +64,8 @@
     this.cols            = cols;
     this.comment         = comment;
     this.fieldDelim      = fieldDelim;
-    this.isSequenceFile    = isSequenceFile;
+    this.inputFormat     = inputFormat;
+    this.outputFormat    = outputFormat;
     this.lineDelim       = lineDelim;
     this.location        = location;
     this.mapKeyDelim     = mapKeyDelim;
@@ -171,13 +173,22 @@
     this.comment = comment;
   }
 
-  @explain(displayName="isSequenceFile")
-  public boolean isSequenceFile() {
-    return isSequenceFile;
+  @explain(displayName="input format")
+  public String getInputFormat() {
+    return inputFormat;
   }
 
-  public void setSequenceFile(boolean isSequenceFile) {
-    this.isSequenceFile = isSequenceFile;
+  public void setInputFormat(String inputFormat) {
+    this.inputFormat = inputFormat;
+  }
+
+  @explain(displayName="output format")
+  public String getOutputFormat() {
+    return outputFormat;
+  }
+
+  public void setOutputFormat(String outputFormat) {
+    this.outputFormat = outputFormat;
   }
 
   @explain(displayName="location")

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_bad_class.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_bad_class.q?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_bad_class.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_bad_class.q Tue Dec 23 15:40:45 2008
@@ -0,0 +1,3 @@
+CREATE TABLE dest1(key INT, value STRING) STORED AS
+  INPUTFORMAT 'ClassDoesNotExist'
+  OUTPUTFORMAT 'java.lang.Void';

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_input.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_input.q?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_input.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_input.q Tue Dec 23 15:40:45 2008
@@ -0,0 +1,8 @@
+CREATE TABLE dest1(key INT, value STRING) STORED AS
+  INPUTFORMAT 'java.lang.Void'
+  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat';
+
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10;
+
+SELECT dest1.* FROM dest1;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_output.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_output.q?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_output.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/fileformat_void_output.q Tue Dec 23 15:40:45 2008
@@ -0,0 +1,6 @@
+CREATE TABLE dest1(key INT, value STRING) STORED AS
+  INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+  OUTPUTFORMAT 'java.lang.Void';
+
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10;

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_sequencefile.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_sequencefile.q?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_sequencefile.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_sequencefile.q Tue Dec 23 15:40:45 2008
@@ -0,0 +1,17 @@
+EXPLAIN
+CREATE TABLE dest1(key INT, value STRING) STORED AS
+  INPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileInputFormat'
+  OUTPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileOutputFormat';
+
+CREATE TABLE dest1(key INT, value STRING) STORED AS
+  INPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileInputFormat'
+  OUTPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileOutputFormat';
+
+DESCRIBE EXTENDED dest1;
+
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10;
+
+SELECT dest1.* FROM dest1;
+
+DROP TABLE dest1;

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_text.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_text.q?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_text.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_text.q Tue Dec 23 15:40:45 2008
@@ -0,0 +1,17 @@
+EXPLAIN
+CREATE TABLE dest1(key INT, value STRING) STORED AS
+  INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat';
+
+CREATE TABLE dest1(key INT, value STRING) STORED AS
+  INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat';
+
+DESCRIBE EXTENDED dest1;
+
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10;
+
+SELECT dest1.* FROM dest1;
+
+DROP TABLE dest1;

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_void.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_void.q?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_void.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/fileformat_void.q Tue Dec 23 15:40:45 2008
@@ -0,0 +1,12 @@
+EXPLAIN
+CREATE TABLE dest1(key INT, value STRING) STORED AS
+  INPUTFORMAT 'java.lang.Void'
+  OUTPUTFORMAT 'java.lang.Void';
+
+CREATE TABLE dest1(key INT, value STRING) STORED AS
+  INPUTFORMAT 'java.lang.Void'
+  OUTPUTFORMAT 'java.lang.Void';
+
+DESCRIBE EXTENDED dest1;
+
+DROP TABLE dest1;

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_bad_class.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_bad_class.q.out?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_bad_class.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_bad_class.q.out Tue Dec 23 15:40:45 2008
@@ -0,0 +1,2 @@
+FAILED: Error in metadata: Class not found: ClassDoesNotExist
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_input.q.out Tue Dec 23 15:40:45 2008
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: line 3:20 Input Format must implement InputFormat dest1

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_output.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_output.q.out?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_output.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/fileformat_void_output.q.out Tue Dec 23 15:40:45 2008
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: line 4:23 Output Format must implement OutputFormat dest1

Added: hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_sequencefile.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_sequencefile.q.out?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_sequencefile.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_sequencefile.q.out Tue Dec 23 15:40:45 2008
@@ -0,0 +1,32 @@
+ABSTRACT SYNTAX TREE:
+  (TOK_CREATETABLE dest1 (TOK_TABCOLLIST (TOK_TABCOL key TOK_INT) (TOK_TABCOL value TOK_STRING)) (TOK_TABLEFILEFORMAT 'org.apache.hadoop.mapred.SequenceFileInputFormat' 'org.apache.hadoop.mapred.SequenceFileOutputFormat'))
+
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Create Table Operator:
+        Create Table
+          columns: key int, value string
+          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+          # buckets: -1
+          output format: org.apache.hadoop.mapred.SequenceFileOutputFormat
+          name: dest1
+          isExternal: false
+
+
+key	int
+value	string
+Detailed Table Information:
+Table(tableName:dest1,dbName:default,owner:dphillips,createTime:1229534167,lastAccessTime:0,retention:0,sd:StorageDescriptor(cols:[FieldSchema(name:key,type:int,comment:null), FieldSchema(name:value,type:string,comment:null)],location:file:/vm/tmp/hive/build/ql/test/data/warehouse/dest1,inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat,outputFormat:org.apache.hadoop.mapred.SequenceFileOutputFormat,compressed:false,numBuckets:-1,serdeInfo:SerDeInfo(name:null,serializationLib:org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe,parameters:{serialization.format=1}),bucketCols:[],sortCols:[],parameters:{}),partitionKeys:[],parameters:{})
+0	val_0
+4	val_4
+8	val_8
+0	val_0
+0	val_0
+5	val_5
+5	val_5
+2	val_2
+5	val_5
+9	val_9

Added: hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_text.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_text.q.out?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_text.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_text.q.out Tue Dec 23 15:40:45 2008
@@ -0,0 +1,32 @@
+ABSTRACT SYNTAX TREE:
+  (TOK_CREATETABLE dest1 (TOK_TABCOLLIST (TOK_TABCOL key TOK_INT) (TOK_TABCOL value TOK_STRING)) (TOK_TABLEFILEFORMAT 'org.apache.hadoop.mapred.TextInputFormat' 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'))
+
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Create Table Operator:
+        Create Table
+          columns: key int, value string
+          input format: org.apache.hadoop.mapred.TextInputFormat
+          # buckets: -1
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          name: dest1
+          isExternal: false
+
+
+key	int
+value	string
+Detailed Table Information:
+Table(tableName:dest1,dbName:default,owner:dphillips,createTime:1229534038,lastAccessTime:0,retention:0,sd:StorageDescriptor(cols:[FieldSchema(name:key,type:int,comment:null), FieldSchema(name:value,type:string,comment:null)],location:file:/vm/tmp/hive/build/ql/test/data/warehouse/dest1,inputFormat:org.apache.hadoop.mapred.TextInputFormat,outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat,compressed:false,numBuckets:-1,serdeInfo:SerDeInfo(name:null,serializationLib:org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe,parameters:{serialization.format=1}),bucketCols:[],sortCols:[],parameters:{}),partitionKeys:[],parameters:{})
+0	val_0
+4	val_4
+8	val_8
+0	val_0
+0	val_0
+5	val_5
+5	val_5
+2	val_2
+5	val_5
+9	val_9

Added: hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_void.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_void.q.out?rev=729176&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_void.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_void.q.out Tue Dec 23 15:40:45 2008
@@ -0,0 +1,22 @@
+ABSTRACT SYNTAX TREE:
+  (TOK_CREATETABLE dest1 (TOK_TABCOLLIST (TOK_TABCOL key TOK_INT) (TOK_TABCOL value TOK_STRING)) (TOK_TABLEFILEFORMAT 'java.lang.Void' 'java.lang.Void'))
+
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Create Table Operator:
+        Create Table
+          columns: key int, value string
+          input format: java.lang.Void
+          # buckets: -1
+          output format: java.lang.Void
+          name: dest1
+          isExternal: false
+
+
+key	int
+value	string
+Detailed Table Information:
+Table(tableName:dest1,dbName:default,owner:dphillips,createTime:1229530913,lastAccessTime:0,retention:0,sd:StorageDescriptor(cols:[FieldSchema(name:key,type:int,comment:null), FieldSchema(name:value,type:string,comment:null)],location:file:/vm/tmp/hive/build/ql/test/data/warehouse/dest1,inputFormat:java.lang.Void,outputFormat:java.lang.Void,compressed:false,numBuckets:-1,serdeInfo:SerDeInfo(name:null,serializationLib:org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe,parameters:{serialization.format=1}),bucketCols:[],sortCols:[],parameters:{}),partitionKeys:[],parameters:{})

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/input15.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/input15.q.out?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/input15.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/input15.q.out Tue Dec 23 15:40:45 2008
@@ -10,10 +10,11 @@
         Create Table
           columns: key int, value string
           field delimiter: 	
+          input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
           name: TEST15
           isExternal: false
-          isSequenceFile: false
 
 
 key	int

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl1.q.out?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl1.q.out Tue Dec 23 15:40:45 2008
@@ -9,9 +9,10 @@
       Create Table Operator:
         Create Table
           columns: key int, value string
+          input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
           name: INPUTDDL1
           isExternal: false
-          isSequenceFile: false
 
 

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl2.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl2.q.out?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl2.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl2.q.out Tue Dec 23 15:40:45 2008
@@ -9,11 +9,12 @@
       Create Table Operator:
         Create Table
           columns: key int, value string
+          input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
           partition columns: ds datetime, country string
           name: INPUTDDL2
           isExternal: false
-          isSequenceFile: false
 
 
 key	int

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl3.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl3.q.out?rev=729176&r1=729175&r2=729176&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl3.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/inputddl3.q.out Tue Dec 23 15:40:45 2008
@@ -10,10 +10,11 @@
         Create Table
           columns: key int, value string
           field delimiter: 	
+          input format: org.apache.hadoop.mapred.TextInputFormat
           # buckets: -1
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
           name: INPUTDDL3
           isExternal: false
-          isSequenceFile: false
 
 
 key	int