You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by nz...@apache.org on 2010/01/26 08:42:06 UTC

svn commit: r903112 - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/io/ ql/src/test/queries/clientpositive/ ql/src/test/results/clientpositive/

Author: nzhang
Date: Tue Jan 26 07:42:06 2010
New Revision: 903112

URL: http://svn.apache.org/viewvc?rev=903112&view=rev
Log:
HIVE-1087:  Let user script write out binary data into a table

Added:
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BinaryRecordReader.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveBinaryOutputFormat.java
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/binary_output_format.q
    hadoop/hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/RecordReader.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveRecordReader.java

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=903112&r1=903111&r2=903112&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Tue Jan 26 07:42:06 2010
@@ -1508,3 +1508,5 @@
     HIVE-1088. RCFileRecordReader's first split will read duplicate rows if
     the split end is < the first sync mark (Yongqiang He via Ning Zhang)
 
+    HIVE-1087. Let user script write out binary data into a table 
+    (Zheng Shao via Ning Zhang)

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BinaryRecordReader.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BinaryRecordReader.java?rev=903112&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BinaryRecordReader.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BinaryRecordReader.java Tue Jan 26 07:42:06 2010
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.LineRecordReader.LineReader;
+
+/**
+ * Read from a binary stream and treat each 1000 bytes (configurable via 
+ * hive.binary.record.max.length) as a record.  The last record before the 
+ * end of stream can have less than 1000 bytes. 
+ */
+public class BinaryRecordReader implements RecordReader {
+
+  private InputStream in;
+  private BytesWritable bytes;
+  private int maxRecordLength;
+
+  public void initialize(InputStream in, Configuration conf, Properties tbl)
+      throws IOException {
+    this.in = in;
+    maxRecordLength = conf.getInt("hive.binary.record.max.length", 1000);
+  }
+
+  public Writable createRow() throws IOException {
+    bytes = new BytesWritable();
+    bytes.setCapacity(maxRecordLength);
+    return bytes;
+  }
+
+  public int next(Writable row) throws IOException {
+    int recordLength = in.read(bytes.get(), 0, maxRecordLength);
+    if (recordLength >= 0) {
+      bytes.setSize(recordLength);
+    }
+    return recordLength;
+  }
+
+  public void close() throws IOException {
+    if (in != null) {
+      in.close();
+    }
+  }
+}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/RecordReader.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/RecordReader.java?rev=903112&r1=903111&r2=903112&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/RecordReader.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/RecordReader.java Tue Jan 26 07:42:06 2010
@@ -32,6 +32,10 @@
 
   public Writable createRow() throws IOException;
 
+  /**
+   * Returns the number of bytes that we consumed.
+   * -1 means end of stream.
+   */
   public int next(Writable row) throws IOException;
 
   public void close() throws IOException;

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveBinaryOutputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveBinaryOutputFormat.java?rev=903112&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveBinaryOutputFormat.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveBinaryOutputFormat.java Tue Jan 26 07:42:06 2010
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Properties;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
+import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * HiveBinaryOutputFormat writes out the values consecutively without any
+ * separators.  It can be used to create a binary data file.
+ */
+public class HiveBinaryOutputFormat<K extends WritableComparable, V extends Writable>
+    extends TextOutputFormat<K, V> implements HiveOutputFormat<K, V> {
+
+  /**
+   * create the final out file, and output row by row. After one row is
+   * appended, a configured row separator is appended
+   * 
+   * @param jc
+   *          the job configuration file
+   * @param outPath
+   *          the final output file to be created
+   * @param valueClass
+   *          the value class used for create
+   * @param isCompressed
+   *          ignored. Currently we don't support compression.
+   * @param tableProperties
+   *          the tableProperties of this file's corresponding table
+   * @param progress
+   *          progress used for status report
+   * @return the RecordWriter
+   */
+  @Override
+  public RecordWriter getHiveRecordWriter(JobConf jc, Path outPath,
+      Class<? extends Writable> valueClass, boolean isCompressed,
+      Properties tableProperties, Progressable progress) throws IOException {
+
+    FileSystem fs = outPath.getFileSystem(jc);
+    final OutputStream outStream = fs.create(outPath);
+    
+    return new RecordWriter() {
+      public void write(Writable r) throws IOException {
+        if (r instanceof Text) {
+          Text tr = (Text) r;
+          outStream.write(tr.getBytes(), 0, tr.getLength());
+        } else {
+          // DynamicSerDe always writes out BytesWritable
+          BytesWritable bw = (BytesWritable) r;
+          outStream.write(bw.get(), 0, bw.getSize());
+        }
+      }
+
+      public void close(boolean abort) throws IOException {
+        outStream.close();
+      }
+    };
+  }
+
+}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveRecordReader.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveRecordReader.java?rev=903112&r1=903111&r2=903112&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveRecordReader.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveRecordReader.java Tue Jan 26 07:42:06 2010
@@ -25,6 +25,11 @@
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.RecordReader;
 
+/**
+ * HiveRecordReader is a simple wrapper on RecordReader.
+ * It allows us to stop reading the data when some global flag
+ * ExecMapper.getDone() is set. 
+ */
 public class HiveRecordReader<K extends WritableComparable, V extends Writable>
     implements RecordReader<K, V> {
 

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/binary_output_format.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/binary_output_format.q?rev=903112&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/binary_output_format.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/binary_output_format.q Tue Jan 26 07:42:06 2010
@@ -0,0 +1,43 @@
+DROP TABLE dest1;
+
+-- Create a table with binary output format
+CREATE TABLE dest1(mydata STRING)
+ROW FORMAT SERDE
+  'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+WITH SERDEPROPERTIES (
+  'serialization.last.column.takes.rest'='true'
+)
+STORED AS
+  INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat';
+
+-- Insert into that table using transform
+EXPLAIN EXTENDED
+INSERT OVERWRITE TABLE dest1
+SELECT TRANSFORM(*)
+  USING 'cat'
+  AS mydata STRING
+    ROW FORMAT SERDE
+      'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+    WITH SERDEPROPERTIES (
+      'serialization.last.column.takes.rest'='true'
+    )
+    RECORDREADER 'org.apache.hadoop.hive.ql.exec.BinaryRecordReader'
+FROM src;
+
+INSERT OVERWRITE TABLE dest1
+SELECT TRANSFORM(*)
+  USING 'cat'
+  AS mydata STRING
+    ROW FORMAT SERDE
+      'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+    WITH SERDEPROPERTIES (
+      'serialization.last.column.takes.rest'='true'
+    )
+    RECORDREADER 'org.apache.hadoop.hive.ql.exec.BinaryRecordReader'
+FROM src;
+
+-- Test the result
+SELECT * FROM dest1;
+
+DROP TABLE dest1;

Added: hadoop/hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out?rev=903112&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out Tue Jan 26 07:42:06 2010
@@ -0,0 +1,813 @@
+PREHOOK: query: DROP TABLE dest1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE dest1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: -- Create a table with binary output format
+CREATE TABLE dest1(mydata STRING)
+ROW FORMAT SERDE
+  'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+WITH SERDEPROPERTIES (
+  'serialization.last.column.takes.rest'='true'
+)
+STORED AS
+  INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat'
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- Create a table with binary output format
+CREATE TABLE dest1(mydata STRING)
+ROW FORMAT SERDE
+  'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+WITH SERDEPROPERTIES (
+  'serialization.last.column.takes.rest'='true'
+)
+STORED AS
+  INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@dest1
+PREHOOK: query: -- Insert into that table using transform
+EXPLAIN EXTENDED
+INSERT OVERWRITE TABLE dest1
+SELECT TRANSFORM(*)
+  USING 'cat'
+  AS mydata STRING
+    ROW FORMAT SERDE
+      'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+    WITH SERDEPROPERTIES (
+      'serialization.last.column.takes.rest'='true'
+    )
+    RECORDREADER 'org.apache.hadoop.hive.ql.exec.BinaryRecordReader'
+FROM src
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Insert into that table using transform
+EXPLAIN EXTENDED
+INSERT OVERWRITE TABLE dest1
+SELECT TRANSFORM(*)
+  USING 'cat'
+  AS mydata STRING
+    ROW FORMAT SERDE
+      'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+    WITH SERDEPROPERTIES (
+      'serialization.last.column.takes.rest'='true'
+    )
+    RECORDREADER 'org.apache.hadoop.hive.ql.exec.BinaryRecordReader'
+FROM src
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF src)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB dest1)) (TOK_SELECT (TOK_SELEXPR (TOK_TRANSFORM (TOK_EXPLIST TOK_ALLCOLREF) TOK_SERDE TOK_RECORDWRITER 'cat' (TOK_SERDE (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY 'serialization.last.column.takes.rest' 'true'))))) (TOK_RECORDREADER 'org.apache.hadoop.hive.ql.exec.BinaryRecordReader') (TOK_TABCOLLIST (TOK_TABCOL mydata TOK_STRING)))))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-4 depends on stages: Stage-1 , consists of Stage-3, Stage-2
+  Stage-3
+  Stage-0 depends on stages: Stage-3, Stage-2
+  Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        src 
+          TableScan
+            alias: src
+            Select Operator
+              expressions:
+                    expr: key
+                    type: string
+                    expr: value
+                    type: string
+              outputColumnNames: _col0, _col1
+              Transform Operator
+                command: cat
+                output info:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    properties:
+                      field.delim 9
+                      columns _col0
+                      serialization.use.json.object true
+                      serialization.format 9
+                      columns.types string
+                      serialization.last.column.takes.rest true
+                File Output Operator
+                  compressed: false
+                  GlobalTableId: 1
+                  directory: file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1286801631/10002
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+                      properties:
+                        name dest1
+                        columns.types string
+                        serialization.ddl struct dest1 { string mydata}
+                        columns mydata
+                        serialization.format 1
+                        serialization.last.column.takes.rest true
+                        bucket_count -1
+                        serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        file.inputformat org.apache.hadoop.mapred.TextInputFormat
+                        file.outputformat org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+                        location file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/dest1
+                        transient_lastDdlTime 1264211336
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: dest1
+      Needs Tagging: false
+      Path -> Alias:
+        file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src [src]
+      Path -> Partition:
+        file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src 
+          Partition
+            base file name: src
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              name src
+              columns.types string:string
+              serialization.ddl struct src { string key, string value}
+              serialization.format 1
+              columns key,value
+              bucket_count -1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              file.inputformat org.apache.hadoop.mapred.TextInputFormat
+              file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              location file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src
+              transient_lastDdlTime 1264211335
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                name src
+                columns.types string:string
+                serialization.ddl struct src { string key, string value}
+                serialization.format 1
+                columns key,value
+                bucket_count -1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                file.inputformat org.apache.hadoop.mapred.TextInputFormat
+                file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                location file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/src
+                transient_lastDdlTime 1264211335
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: src
+            name: src
+
+  Stage: Stage-4
+    Conditional Operator
+
+  Stage: Stage-3
+    Move Operator
+      files:
+          hdfs directory: true
+          source: file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1286801631/10002
+          destination: file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1065869242/10000
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: true
+          source: file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1065869242/10000
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+              properties:
+                name dest1
+                columns.types string
+                serialization.ddl struct dest1 { string mydata}
+                columns mydata
+                serialization.format 1
+                serialization.last.column.takes.rest true
+                bucket_count -1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                file.inputformat org.apache.hadoop.mapred.TextInputFormat
+                file.outputformat org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+                location file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/dest1
+                transient_lastDdlTime 1264211336
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: dest1
+          tmp directory: file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1065869242/10001
+
+  Stage: Stage-2
+    Map Reduce
+      Alias -> Map Operator Tree:
+        file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1286801631/10002 
+            Reduce Output Operator
+              sort order: 
+              Map-reduce partition columns:
+                    expr: rand()
+                    type: double
+              tag: -1
+              value expressions:
+                    expr: mydata
+                    type: string
+      Needs Tagging: false
+      Path -> Alias:
+        file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1286801631/10002 [file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1286801631/10002]
+      Path -> Partition:
+        file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1286801631/10002 
+          Partition
+            base file name: 10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+            properties:
+              name dest1
+              columns.types string
+              serialization.ddl struct dest1 { string mydata}
+              columns mydata
+              serialization.format 1
+              serialization.last.column.takes.rest true
+              bucket_count -1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              file.inputformat org.apache.hadoop.mapred.TextInputFormat
+              file.outputformat org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+              location file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/dest1
+              transient_lastDdlTime 1264211336
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+              properties:
+                name dest1
+                columns.types string
+                serialization.ddl struct dest1 { string mydata}
+                columns mydata
+                serialization.format 1
+                serialization.last.column.takes.rest true
+                bucket_count -1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                file.inputformat org.apache.hadoop.mapred.TextInputFormat
+                file.outputformat org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+                location file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/dest1
+                transient_lastDdlTime 1264211336
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: dest1
+            name: dest1
+      Reduce Operator Tree:
+        Extract
+          File Output Operator
+            compressed: false
+            GlobalTableId: 0
+            directory: file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1065869242/10000
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+                properties:
+                  name dest1
+                  columns.types string
+                  serialization.ddl struct dest1 { string mydata}
+                  serialization.format 1
+                  columns mydata
+                  serialization.last.column.takes.rest true
+                  bucket_count -1
+                  serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  file.inputformat org.apache.hadoop.mapred.TextInputFormat
+                  location file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/dest1
+                  file.outputformat org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+                  transient_lastDdlTime 1264211336
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                name: dest1
+
+
+PREHOOK: query: INSERT OVERWRITE TABLE dest1
+SELECT TRANSFORM(*)
+  USING 'cat'
+  AS mydata STRING
+    ROW FORMAT SERDE
+      'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+    WITH SERDEPROPERTIES (
+      'serialization.last.column.takes.rest'='true'
+    )
+    RECORDREADER 'org.apache.hadoop.hive.ql.exec.BinaryRecordReader'
+FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@dest1
+POSTHOOK: query: INSERT OVERWRITE TABLE dest1
+SELECT TRANSFORM(*)
+  USING 'cat'
+  AS mydata STRING
+    ROW FORMAT SERDE
+      'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+    WITH SERDEPROPERTIES (
+      'serialization.last.column.takes.rest'='true'
+    )
+    RECORDREADER 'org.apache.hadoop.hive.ql.exec.BinaryRecordReader'
+FROM src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@dest1
+PREHOOK: query: -- Test the result
+SELECT * FROM dest1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dest1
+PREHOOK: Output: file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1904733215/10000
+POSTHOOK: query: -- Test the result
+SELECT * FROM dest1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dest1
+POSTHOOK: Output: file:/data/users/zshao/svnhive_hadoop/trunk/VENDOR.hive/trunk/build/ql/scratchdir/1904733215/10000
+238	val_238
+86	val_86
+311	val_311
+27	val_27
+165	val_165
+409	val_409
+255	val_255
+278	val_278
+98	val_98
+484	val_484
+265	val_265
+193	val_193
+401	val_401
+150	val_150
+273	val_273
+224	val_224
+369	val_369
+66	val_66
+128	val_128
+213	val_213
+146	val_146
+406	val_406
+429	val_429
+374	val_374
+152	val_152
+469	val_469
+145	val_145
+495	val_495
+37	val_37
+327	val_327
+281	val_281
+277	val_277
+209	val_209
+15	val_15
+82	val_82
+403	val_403
+166	val_166
+417	val_417
+430	val_430
+252	val_252
+292	val_292
+219	val_219
+287	val_287
+153	val_153
+193	val_193
+338	val_338
+446	val_446
+459	val_459
+394	val_394
+237	val_237
+482	val_482
+174	val_174
+413	val_413
+494	val_494
+207	val_207
+199	val_199
+466	val_466
+208	val_208
+174	val_174
+399	val_399
+396	val_396
+247	val_247
+417	val_417
+489	val_489
+162	val_162
+377	val_377
+397	val_397
+309	val_309
+365	val_365
+266	val_266
+439	val_439
+342	val_342
+367	val_367
+325	val_325
+167	val_167
+195	val_195
+475	val_475
+17	val_17
+113	val_113
+155	val_155
+203	val_203
+339	val_339
+0	val_0
+455	val_455
+128	val_128
+311	val_311
+316	val_316
+57	val_57
+302	val_302
+205	val_205
+149	val_149
+438	val_438
+345	val_345
+129	val_129
+170	val_170
+20	val_20
+489	val_489
+157	val_157
+378	val_378
+221	val_221
+92	val_92
+111	val_111
+47	val_47
+72	val_72
+4	val_4
+280	val_280
+35	val_35
+427	val_427
+277	val_277
+208	val_208
+356	val_356
+399	val_399
+169	val_169
+382	val_382
+498	val_498
+125	val_125
+386	val_386
+437	val_437
+469	val_469
+192	val_192
+286	val_286
+187	val_187
+176	val_176
+54	val_54
+459	val_459
+51	val_51
+138	val_138
+103	val_103
+239	val_239
+213	val_213
+216	val_216
+430	val_430
+278	val_278
+176	val_176
+289	val_289
+221	val_221
+65	val_65
+318	val_318
+332	val_332
+311	val_311
+275	val_275
+137	val_137
+241	val_241
+83	val_83
+333	val_333
+180	val_180
+284	val_284
+12	val_12
+230	val_230
+181	val_181
+67	val_67
+260	val_260
+404	val_404
+384	val_384
+489	val_489
+353	val_353
+373	val_373
+272	val_272
+138	val_138
+217	val_217
+84	val_84
+348	val_348
+466	val_466
+58	val_58
+8	val_8
+411	val_411
+230	val_230
+208	val_208
+348	val_348
+24	val_24
+463	val_463
+431	val_431
+179	val_179
+172	val_172
+42	val_42
+129	val_129
+158	val_158
+119	val_119
+496	val_496
+0	val_0
+322	val_322
+197	val_197
+468	val_468
+393	val_393
+454	val_454
+100	val_100
+298	val_298
+199	val_199
+191	val_191
+418	val_418
+96	val_96
+26	val_26
+165	val_165
+327	val_327
+230	val_230
+205	val_205
+120	val_120
+131	val_131
+51	val_51
+404	val_404
+43	val_43
+436	val_436
+156	val_156
+469	val_469
+468	val_468
+308	val_308
+95	val_95
+196	val_196
+288	val_288
+481	val_481
+457	val_457
+98	val_98
+282	val_282
+197	val_197
+187	val_187
+318	val_318
+318	val_318
+409	val_409
+470	val_470
+137	val_137
+369	val_369
+316	val_316
+169	val_169
+413	val_413
+85	val_85
+77	val_77
+0	val_0
+490	val_490
+87	val_87
+364	val_364
+179	val_179
+118	val_118
+134	val_134
+395	val_395
+282	val_282
+138	val_138
+238	val_238
+419	val_419
+15	val_15
+118	val_118
+72	val_72
+90	val_90
+307	val_307
+19	val_19
+435	val_435
+10	val_10
+277	val_277
+273	val_273
+306	val_306
+224	val_224
+309	val_309
+389	val_389
+327	val_327
+242	val_242
+369	val_369
+392	val_392
+272	val_272
+331	val_331
+401	val_401
+242	val_242
+452	val_452
+177	val_177
+226	val_226
+5	val_5
+497	val_497
+402	val_402
+396	val_396
+317	val_317
+395	val_395
+58	val_58
+35	val_35
+336	val_336
+95	val_95
+11	val_11
+168	val_168
+34	val_34
+229	val_229
+233	val_233
+143	val_143
+472	val_472
+322	val_322
+498	val_498
+160	val_160
+195	val_195
+42	val_42
+321	val_321
+430	val_430
+119	val_119
+489	val_489
+458	val_458
+78	val_78
+76	val_76
+41	val_41
+223	val_223
+492	val_492
+149	val_149
+449	val_449
+218	val_218
+228	val_228
+138	val_138
+453	val_453
+30	val_30
+209	val_209
+64	val_64
+468	val_468
+76	val_76
+74	val_74
+342	val_342
+69	val_69
+230	val_230
+33	val_33
+368	val_368
+103	val_103
+296	val_296
+113	val_113
+216	val_216
+367	val_367
+344	val_344
+167	val_167
+274	val_274
+219	val_219
+239	val_239
+485	val_485
+116	val_116
+223	val_223
+256	val_256
+263	val_263
+70	val_70
+487	val_487
+480	val_480
+401	val_401
+288	val_288
+191	val_191
+5	val_5
+244	val_244
+438	val_438
+128	val_128
+467	val_467
+432	val_432
+202	val_202
+316	val_316
+229	val_229
+469	val_469
+463	val_463
+280	val_280
+2	val_2
+35	val_35
+283	val_283
+331	val_331
+235	val_235
+80	val_80
+44	val_44
+193	val_193
+321	val_321
+335	val_335
+104	val_104
+466	val_466
+366	val_366
+175	val_175
+403	val_403
+483	val_483
+53	val_53
+105	val_105
+257	val_257
+406	val_406
+409	val_409
+190	val_190
+406	val_406
+401	val_401
+114	val_114
+258	val_258
+90	val_90
+203	val_203
+262	val_262
+348	val_348
+424	val_424
+12	val_12
+396	val_396
+201	val_201
+217	val_217
+164	val_164
+431	val_431
+454	val_454
+478	val_478
+298	val_298
+125	val_125
+431	val_431
+164	val_164
+424	val_424
+187	val_187
+382	val_382
+5	val_5
+70	val_70
+397	val_397
+480	val_480
+291	val_291
+24	val_24
+351	val_351
+255	val_255
+104	val_104
+70	val_70
+163	val_163
+438	val_438
+119	val_119
+414	val_414
+200	val_200
+491	val_491
+237	val_237
+439	val_439
+360	val_360
+248	val_248
+479	val_479
+305	val_305
+417	val_417
+199	val_199
+444	val_444
+120	val_120
+429	val_429
+169	val_169
+443	val_443
+323	val_323
+325	val_325
+277	val_277
+230	val_230
+478	val_478
+178	val_178
+468	val_468
+310	val_310
+317	val_317
+333	val_333
+493	val_493
+460	val_460
+207	val_207
+249	val_249
+265	val_265
+480	val_480
+83	val_83
+136	val_136
+353	val_353
+172	val_172
+214	val_214
+462	val_462
+233	val_233
+406	val_406
+133	val_133
+175	val_175
+189	val_189
+454	val_454
+375	val_375
+401	val_401
+421	val_421
+407	val_407
+384	val_384
+256	val_256
+26	val_26
+134	val_134
+67	val_67
+384	val_384
+379	val_379
+18	val_18
+462	val_462
+492	val_492
+100	val_100
+298	val_298
+9	val_9
+341	val_341
+498	val_498
+146	val_146
+458	val_458
+362	val_362
+186	val_186
+285	val_285
+348	val_348
+167	val_167
+18	val_18
+273	val_273
+183	val_183
+281	val_281
+344	val_344
+97	val_97
+469	val_469
+315	val_315
+84	val_84
+28	val_28
+37	val_37
+448	val_448
+152	val_152
+348	val_348
+307	val_307
+194	val_194
+414	val_414
+477	val_477
+222	val_222
+126	val_126
+90	val_90
+169	val_169
+403	val_403
+400	val_400
+200	val_200
+97	val_97
+PREHOOK: query: DROP TABLE dest1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE dest1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: default@dest1