You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by cw...@apache.org on 2011/05/23 22:34:59 UTC

svn commit: r1126726 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/metadata/ test/org/apache/hadoop/hive/ql/ test/queries/clientpositive/ test/results/clientpositive/

Author: cws
Date: Mon May 23 20:34:58 2011
New Revision: 1126726

URL: http://svn.apache.org/viewvc?rev=1126726&view=rev
Log:
HIVE-2117. Insert overwrite ignoring partition location (Patrick Hunt via cws)

Added:
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/BaseTestQueries.java   (with props)
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java   (with props)
    hive/trunk/ql/src/test/queries/clientpositive/alter5.q
    hive/trunk/ql/src/test/results/clientpositive/alter5.q.out
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1126726&r1=1126725&r2=1126726&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Mon May 23 20:34:58 2011
@@ -1084,7 +1084,6 @@ public class Hive {
         oldPartPath = oldPart.getPartitionPath();
       }
 
-
       Path newPartPath = null;
 
       if (inheritTableSpecs) {
@@ -1092,6 +1091,21 @@ public class Hive {
             Warehouse.makePartPath(partSpec));
         newPartPath = new Path(loadPath.toUri().getScheme(), loadPath.toUri().getAuthority(),
             partPath.toUri().getPath());
+
+        if(oldPart != null) {
+          /*
+           * If we are moving the partition across filesystem boundaries
+           * inherit from the table properties. Otherwise (same filesystem) use the
+           * original partition location.
+           *
+           * See: HIVE-1707 and HIVE-2117 for background
+           */
+          FileSystem oldPartPathFS = oldPartPath.getFileSystem(getConf());
+          FileSystem loadPathFS = loadPath.getFileSystem(getConf());
+          if (oldPartPathFS.equals(loadPathFS)) {
+            newPartPath = oldPartPath;
+          }
+        }
       } else {
         newPartPath = oldPartPath;
       }

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/BaseTestQueries.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/BaseTestQueries.java?rev=1126726&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/BaseTestQueries.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/BaseTestQueries.java Mon May 23 20:34:58 2011
@@ -0,0 +1,49 @@
+/**
+ * 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;
+
+import java.io.File;
+
+import junit.framework.TestCase;
+
+/**
+ * Base class for testing queries.
+ */
+public abstract class BaseTestQueries extends TestCase {
+
+  protected final String inpDir = System
+      .getProperty("ql.test.query.clientpositive.dir");
+  protected final String resDir = System
+      .getProperty("ql.test.results.clientpositive.dir");
+  protected final String logDir = System.getProperty("test.log.dir")
+      + "/clientpositive";
+
+  /**
+   * Create a file for each test name in the inpDir.
+   * @param testNames
+   * @return files corresponding to each test name
+   */
+  protected File[] setupQFiles(String[] testNames) {
+    File[] qfiles = new File[testNames.length];
+    for (int i = 0; i < testNames.length; i++) {
+      qfiles[i] = new File(inpDir, testNames[i]);
+    }
+    return qfiles;
+  }
+}

Propchange: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/BaseTestQueries.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1126726&r1=1126725&r2=1126726&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java Mon May 23 20:34:58 2011
@@ -42,8 +42,6 @@ import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import junit.framework.Test;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
@@ -95,8 +93,8 @@ public class QTestUtil {
 
 
   private final String testFiles;
-  private final String outDir;
-  private final String logDir;
+  protected final String outDir;
+  protected final String logDir;
   private final TreeMap<String, String> qMap;
   private final Set<String> qSkipSet;
   public static final HashSet<String> srcTables = new HashSet<String>
@@ -111,7 +109,7 @@ public class QTestUtil {
   private Driver drv;
   private SemanticAnalyzer sem;
   private FileSystem fs;
-  private boolean overWrite;
+  protected final boolean overWrite;
   private CliDriver cliDriver;
   private MiniMRCluster mr = null;
   private HadoopShims.MiniDFSShim dfs = null;
@@ -240,9 +238,10 @@ public class QTestUtil {
         .replace("c:", "");
 
     String ow = System.getProperty("test.output.overwrite");
-    overWrite = false;
     if ((ow != null) && ow.equalsIgnoreCase("true")) {
       overWrite = true;
+    } else {
+      overWrite = false;
     }
 
     setup = new QTestSetup();
@@ -1092,85 +1091,108 @@ public class QTestUtil {
   }
 
   /**
-   * executes a set of query files either in sequence or in parallel. Uses
-   * QTestUtil to do so
+   * Setup to execute a set of query files. Uses QTestUtil to do so.
    *
    * @param qfiles
    *          array of input query files containing arbitrary number of hive
    *          queries
-   * @param resDirs
-   *          array of output directories one corresponding to each input query
-   *          file
-   * @param mt
-   *          whether to run in multithreaded mode or not
-   * @return true if all the query files were executed successfully, else false
+   * @param resDir
+   *          output directory
+   * @param logDir
+   *          log directory
+   * @return one QTestUtil for each query file
+   */
+  public static QTestUtil[] queryListRunnerSetup(File[] qfiles, String resDir,
+      String logDir) throws Exception
+  {
+    QTestUtil[] qt = new QTestUtil[qfiles.length];
+    for (int i = 0; i < qfiles.length; i++) {
+      qt[i] = new QTestUtil(resDir, logDir, false, "0.20");
+      qt[i].addFile(qfiles[i]);
+      qt[i].clearTestSideEffects();
+    }
+
+    return qt;
+  }
+
+  /**
+   * Executes a set of query files in sequence.
    *
-   *         In multithreaded mode each query file is run in a separate thread.
-   *         the caller has to arrange that different query files do not collide
-   *         (in terms of destination tables)
+   * @param qfiles
+   *          array of input query files containing arbitrary number of hive
+   *          queries
+   * @param qt
+   *          array of QTestUtils, one per qfile
+   * @return true if all queries passed, false otw
    */
-  public static boolean queryListRunner(File[] qfiles, String[] resDirs,
-                                        String[] logDirs, boolean mt, Test test) {
+  public static boolean queryListRunnerSingleThreaded(File[] qfiles, QTestUtil[] qt)
+    throws Exception
+  {
+    boolean failed = false;
+    qt[0].cleanUp();
+    qt[0].createSources();
+    for (int i = 0; i < qfiles.length && !failed; i++) {
+      qt[i].clearTestSideEffects();
+      qt[i].cliInit(qfiles[i].getName(), false);
+      qt[i].executeClient(qfiles[i].getName());
+      int ecode = qt[i].checkCliDriverResults(qfiles[i].getName());
+      if (ecode != 0) {
+        failed = true;
+        System.err.println("Test " + qfiles[i].getName()
+            + " results check failed with error code " + ecode);
+        outputTestFailureHelpMessage();
+      }
+      qt[i].clearPostTestEffects();
+    }
+    return (!failed);
+  }
 
-    assert (qfiles.length == resDirs.length);
-    assert (qfiles.length == logDirs.length);
+  /**
+   * Executes a set of query files parallel.
+   *
+   * Each query file is run in a separate thread. The caller has to arrange
+   * that different query files do not collide (in terms of destination tables)
+   *
+   * @param qfiles
+   *          array of input query files containing arbitrary number of hive
+   *          queries
+   * @param qt
+   *          array of QTestUtils, one per qfile
+   * @return true if all queries passed, false otw
+   *
+   */
+  public static boolean queryListRunnerMultiThreaded(File[] qfiles, QTestUtil[] qt)
+    throws Exception
+  {
     boolean failed = false;
-    try {
-      QTestUtil[] qt = new QTestUtil[qfiles.length];
-      QTestSetup[] qsetup = new QTestSetup[qfiles.length];
-      for (int i = 0; i < qfiles.length; i++) {
-        qt[i] = new QTestUtil(resDirs[i], logDirs[i], false, "0.20");
-        qt[i].addFile(qfiles[i]);
-        qt[i].clearTestSideEffects();
-      }
-
-      if (mt) {
-        // in multithreaded mode - do cleanup/initialization just once
-
-        qt[0].cleanUp();
-        qt[0].createSources();
-        qt[0].clearTestSideEffects();
-
-        QTRunner[] qtRunners = new QTestUtil.QTRunner[qfiles.length];
-        Thread[] qtThread = new Thread[qfiles.length];
-
-        for (int i = 0; i < qfiles.length; i++) {
-          qtRunners[i] = new QTestUtil.QTRunner(qt[i], qfiles[i].getName());
-          qtThread[i] = new Thread(qtRunners[i]);
-        }
 
-        for (int i = 0; i < qfiles.length; i++) {
-          qtThread[i].start();
-        }
+    // in multithreaded mode - do cleanup/initialization just once
 
-        for (int i = 0; i < qfiles.length; i++) {
-          qtThread[i].join();
-          int ecode = qt[i].checkCliDriverResults(qfiles[i].getName());
-          if (ecode != 0) {
-            failed = true;
-            System.err.println("Test " + qfiles[i].getName()
-                + " results check failed with error code " + ecode);
-            outputTestFailureHelpMessage();
-          }
-        }
+    qt[0].cleanUp();
+    qt[0].createSources();
+    qt[0].clearTestSideEffects();
 
-      } else {
+    QTRunner[] qtRunners = new QTestUtil.QTRunner[qfiles.length];
+    Thread[] qtThread = new Thread[qfiles.length];
 
-        for (int i = 0; i < qfiles.length && !failed; i++) {
-          qt[i].cliInit(qfiles[i].getName());
-          qt[i].executeClient(qfiles[i].getName());
-          int ecode = qt[i].checkCliDriverResults(qfiles[i].getName());
-          if (ecode != 0) {
-            failed = true;
-            System.err.println("Test " + qfiles[i].getName()
-                + " results check failed with error code " + ecode);
-            outputTestFailureHelpMessage();
-          }
-        }
+    for (int i = 0; i < qfiles.length; i++) {
+      qtRunners[i] = new QTestUtil.QTRunner(qt[i], qfiles[i].getName());
+      qtThread[i] = new Thread(qtRunners[i]);
+    }
+
+    for (int i = 0; i < qfiles.length; i++) {
+      qtThread[i].start();
+    }
+
+    for (int i = 0; i < qfiles.length; i++) {
+      qtThread[i].join();
+      int ecode = qt[i].checkCliDriverResults(qfiles[i].getName());
+      if (ecode != 0) {
+        failed = true;
+        System.err.println("Test " + qfiles[i].getName()
+            + " results check failed with error code " + ecode);
+        outputTestFailureHelpMessage();
       }
-    } catch (Exception e) {
-      e.printStackTrace();
-      return false;
     }
     return (!failed);
   }

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java?rev=1126726&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java Mon May 23 20:34:58 2011
@@ -0,0 +1,108 @@
+/**
+ * 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;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Suite for testing location. e.g. if "alter table alter partition
+ * location" is run, do the partitions end up in the correct location.
+ *
+ *  This is a special case of the regular queries as paths are typically
+ *  ignored.
+ */
+public class TestLocationQueries extends BaseTestQueries {
+  /**
+   * Our own checker - validate the location of the partition.
+   */
+  public static class CheckResults extends QTestUtil {
+    private final String locationSubdir;
+
+    /**
+     * Validate only that the location is correct.
+     * @return non-zero if it failed
+     */
+    @Override
+    public int checkCliDriverResults(String tname) throws Exception {
+      File logFile = new File(logDir, tname + ".out");
+
+      int failedCount = 0;
+      FileReader fr = new FileReader(logFile);
+      BufferedReader in = new BufferedReader(fr);
+      try {
+        String line;
+        int locationCount = 0;
+        Pattern p = Pattern.compile("location:([^,)]+)");
+        while((line = in.readLine()) != null) {
+          Matcher m = p.matcher(line);
+          if (m.find()) {
+            File f = new File(m.group(1));
+            if (!f.getName().equals(locationSubdir)) {
+              failedCount++;
+            }
+            locationCount++;
+          }
+        }
+        // we always have to find at least one location, otw the test is useless
+        if (locationCount == 0) {
+          return Integer.MAX_VALUE;
+        }
+      } finally {
+        in.close();
+      }
+
+      return failedCount;
+    }
+
+    public CheckResults(String outDir, String logDir, boolean miniMr,
+        String hadoopVer, String locationSubdir)
+      throws Exception
+    {
+      super(outDir, logDir, miniMr, hadoopVer);
+      this.locationSubdir = locationSubdir;
+    }
+  }
+
+  /**
+   * Verify that the location of the partition is valid. In this case
+   * the path should end in "parta" and not "dt=a" (the default).
+   *
+   */
+  public void testAlterTablePartitionLocation_alter5() throws Exception {
+    String[] testNames = new String[] {"alter5.q"};
+
+    File[] qfiles = setupQFiles(testNames);
+
+    QTestUtil[] qt = new QTestUtil[qfiles.length];
+    for (int i = 0; i < qfiles.length; i++) {
+      qt[i] = new CheckResults(resDir, logDir, false, "0.20", "parta");
+      qt[i].addFile(qfiles[i]);
+      qt[i].clearTestSideEffects();
+    }
+
+    boolean success = QTestUtil.queryListRunnerSingleThreaded(qfiles, qt);
+    if (!success) {
+      fail("One or more queries failed");
+    }
+  }
+}

Propchange: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java?rev=1126726&r1=1126725&r2=1126726&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java Mon May 23 20:34:58 2011
@@ -20,33 +20,19 @@ package org.apache.hadoop.hive.ql;
 
 import java.io.File;
 
-import junit.framework.TestCase;
-
 /**
  * Suite for testing running of queries in multi-threaded mode.
  */
-public class TestMTQueries extends TestCase {
-
-  private final String inpDir = System
-      .getProperty("ql.test.query.clientpositive.dir");
-  private final String resDir = System
-      .getProperty("ql.test.results.clientpositive.dir");
-  private final String logDir = System.getProperty("test.log.dir")
-      + "/clientpositive";
+public class TestMTQueries extends BaseTestQueries {
 
   public void testMTQueries1() throws Exception {
     String[] testNames = new String[] {"join1.q", "join2.q", "groupby1.q",
         "groupby2.q", "join3.q", "input1.q", "input19.q"};
-    String[] logDirs = new String[testNames.length];
-    String[] resDirs = new String[testNames.length];
-    File[] qfiles = new File[testNames.length];
-    for (int i = 0; i < resDirs.length; i++) {
-      logDirs[i] = logDir;
-      resDirs[i] = resDir;
-      qfiles[i] = new File(inpDir, testNames[i]);
-    }
 
-    boolean success = QTestUtil.queryListRunner(qfiles, resDirs, logDirs, true, this);
+    File[] qfiles = setupQFiles(testNames);
+
+    QTestUtil[] qts = QTestUtil.queryListRunnerSetup(qfiles, resDir, logDir);
+    boolean success = QTestUtil.queryListRunnerMultiThreaded(qfiles, qts);
     if (!success) {
       fail("One or more queries failed");
     }

Added: hive/trunk/ql/src/test/queries/clientpositive/alter5.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/alter5.q?rev=1126726&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/alter5.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/alter5.q Mon May 23 20:34:58 2011
@@ -0,0 +1,45 @@
+--
+-- Added to validate the fix for HIVE-2117 - explicit partition location
+--
+
+create table alter5_src ( col1 string ) stored as textfile ;
+load data local inpath '../data/files/test.dat' overwrite into table alter5_src ;
+
+create table alter5 ( col1 string ) partitioned by (dt string);
+
+--
+-- Here's the interesting bit for HIVE-2117 - partition subdir should be
+-- named "parta".
+--
+alter table alter5 add partition (dt='a') location 'parta';
+
+describe extended alter5 partition (dt='a');
+
+insert overwrite table alter5 partition (dt='a') select col1 from alter5_src ;
+select * from alter5 where dt='a';
+
+describe extended alter5 partition (dt='a');
+
+-- Cleanup
+DROP TABLE alter5_src;
+DROP TABLE alter5;
+SHOW TABLES;
+
+-- With non-default Database
+
+CREATE DATABASE alter5_db;
+USE alter5_db;
+SHOW TABLES;
+
+create table alter5_src ( col1 string ) stored as textfile ;
+load data local inpath '../data/files/test.dat' overwrite into table alter5_src ;
+
+create table alter5 ( col1 string ) partitioned by (dt string);
+alter table alter5 add partition (dt='a') location 'parta';
+
+describe extended alter5 partition (dt='a');
+
+insert overwrite table alter5 partition (dt='a') select col1 from alter5_src ;
+select * from alter5 where dt='a';
+
+describe extended alter5 partition (dt='a');

Added: hive/trunk/ql/src/test/results/clientpositive/alter5.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/alter5.q.out?rev=1126726&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/alter5.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/alter5.q.out Mon May 23 20:34:58 2011
@@ -0,0 +1,204 @@
+PREHOOK: query: --
+-- Added to validate the fix for HIVE-2117 - explicit partition location
+--
+
+create table alter5_src ( col1 string ) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: --
+-- Added to validate the fix for HIVE-2117 - explicit partition location
+--
+
+create table alter5_src ( col1 string ) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@alter5_src
+PREHOOK: query: load data local inpath '../data/files/test.dat' overwrite into table alter5_src
+PREHOOK: type: LOAD
+PREHOOK: Output: default@alter5_src
+POSTHOOK: query: load data local inpath '../data/files/test.dat' overwrite into table alter5_src
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@alter5_src
+PREHOOK: query: create table alter5 ( col1 string ) partitioned by (dt string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table alter5 ( col1 string ) partitioned by (dt string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@alter5
+PREHOOK: query: --
+-- Here's the interesting bit for HIVE-2117 - partition subdir should be
+-- named "parta".
+--
+alter table alter5 add partition (dt='a') location 'parta'
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: default@alter5
+POSTHOOK: query: --
+-- Here's the interesting bit for HIVE-2117 - partition subdir should be
+-- named "parta".
+--
+alter table alter5 add partition (dt='a') location 'parta'
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Input: default@alter5
+POSTHOOK: Output: default@alter5@dt=a
+PREHOOK: query: describe extended alter5 partition (dt='a')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: describe extended alter5 partition (dt='a')
+POSTHOOK: type: DESCTABLE
+col1	string	
+dt	string	
+	 	 
+Detailed Partition Information	Partition(values:[a], dbName:default, tableName:alter5, createTime:1304626469, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null)], location:pfile:/home/phunt/dev/hive/build/ql/test/data/warehouse/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), parameters:{transient_lastDdlTime=1304626469})	
+PREHOOK: query: insert overwrite table alter5 partition (dt='a') select col1 from alter5_src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alter5_src
+PREHOOK: Output: default@alter5@dt=a
+POSTHOOK: query: insert overwrite table alter5 partition (dt='a') select col1 from alter5_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alter5_src
+POSTHOOK: Output: default@alter5@dt=a
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: select * from alter5 where dt='a'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alter5@dt=a
+PREHOOK: Output: file:/tmp/phunt/hive_2011-05-05_13-14-37_578_1796477555405644183/-mr-10000
+POSTHOOK: query: select * from alter5 where dt='a'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alter5@dt=a
+POSTHOOK: Output: file:/tmp/phunt/hive_2011-05-05_13-14-37_578_1796477555405644183/-mr-10000
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+1	a
+2	a
+3	a
+4	a
+5	a
+6	a
+PREHOOK: query: describe extended alter5 partition (dt='a')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: describe extended alter5 partition (dt='a')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+col1	string	
+dt	string	
+	 	 
+Detailed Partition Information	Partition(values:[a], dbName:default, tableName:alter5, createTime:1304626469, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null)], location:pfile:/home/phunt/dev/hive/build/ql/test/data/warehouse/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), parameters:{numFiles=1, transient_lastDdlTime=1304626476, numRows=6, totalSize=12})	
+PREHOOK: query: -- Cleanup
+DROP TABLE alter5_src
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@alter5_src
+PREHOOK: Output: default@alter5_src
+POSTHOOK: query: -- Cleanup
+DROP TABLE alter5_src
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@alter5_src
+POSTHOOK: Output: default@alter5_src
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: DROP TABLE alter5
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@alter5
+PREHOOK: Output: default@alter5
+POSTHOOK: query: DROP TABLE alter5
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@alter5
+POSTHOOK: Output: default@alter5
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: SHOW TABLES
+PREHOOK: type: SHOWTABLES
+POSTHOOK: query: SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+src
+src1
+src_json
+src_sequencefile
+src_thrift
+srcbucket
+srcbucket2
+srcpart
+PREHOOK: query: -- With non-default Database
+
+CREATE DATABASE alter5_db
+PREHOOK: type: CREATEDATABASE
+POSTHOOK: query: -- With non-default Database
+
+CREATE DATABASE alter5_db
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: USE alter5_db
+PREHOOK: type: SWITCHDATABASE
+POSTHOOK: query: USE alter5_db
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: SHOW TABLES
+PREHOOK: type: SHOWTABLES
+POSTHOOK: query: SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: create table alter5_src ( col1 string ) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table alter5_src ( col1 string ) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: alter5_db@alter5_src
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: load data local inpath '../data/files/test.dat' overwrite into table alter5_src
+PREHOOK: type: LOAD
+PREHOOK: Output: alter5_db@alter5_src
+POSTHOOK: query: load data local inpath '../data/files/test.dat' overwrite into table alter5_src
+POSTHOOK: type: LOAD
+POSTHOOK: Output: alter5_db@alter5_src
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: create table alter5 ( col1 string ) partitioned by (dt string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table alter5 ( col1 string ) partitioned by (dt string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: alter5_db@alter5
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: alter table alter5 add partition (dt='a') location 'parta'
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: alter5_db@alter5
+POSTHOOK: query: alter table alter5 add partition (dt='a') location 'parta'
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Input: alter5_db@alter5
+POSTHOOK: Output: alter5_db@alter5@dt=a
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: describe extended alter5 partition (dt='a')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: describe extended alter5 partition (dt='a')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+col1	string	
+dt	string	
+	 	 
+Detailed Partition Information	Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1304626483, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null)], location:pfile:/home/phunt/dev/hive/build/ql/test/data/warehouse/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), parameters:{transient_lastDdlTime=1304626483})	
+PREHOOK: query: insert overwrite table alter5 partition (dt='a') select col1 from alter5_src
+PREHOOK: type: QUERY
+PREHOOK: Input: alter5_db@alter5_src
+PREHOOK: Output: alter5_db@alter5@dt=a
+POSTHOOK: query: insert overwrite table alter5 partition (dt='a') select col1 from alter5_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: alter5_db@alter5_src
+POSTHOOK: Output: alter5_db@alter5@dt=a
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+PREHOOK: query: select * from alter5 where dt='a'
+PREHOOK: type: QUERY
+PREHOOK: Input: alter5_db@alter5@dt=a
+PREHOOK: Output: file:/tmp/phunt/hive_2011-05-05_13-14-49_883_4190579771388965111/-mr-10000
+POSTHOOK: query: select * from alter5 where dt='a'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: alter5_db@alter5@dt=a
+POSTHOOK: Output: file:/tmp/phunt/hive_2011-05-05_13-14-49_883_4190579771388965111/-mr-10000
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+1	a
+2	a
+3	a
+4	a
+5	a
+6	a
+PREHOOK: query: describe extended alter5 partition (dt='a')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: describe extended alter5 partition (dt='a')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+POSTHOOK: Lineage: alter5 PARTITION(dt=a).col1 SIMPLE [(alter5_src)alter5_src.FieldSchema(name:col1, type:string, comment:null), ]
+col1	string	
+dt	string	
+	 	 
+Detailed Partition Information	Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1304626483, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null)], location:pfile:/home/phunt/dev/hive/build/ql/test/data/warehouse/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), parameters:{numFiles=1, transient_lastDdlTime=1304626489, numRows=6, totalSize=12})