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 2010/04/01 20:24:14 UTC

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

Author: zshao
Date: Thu Apr  1 18:24:14 2010
New Revision: 930059

URL: http://svn.apache.org/viewvc?rev=930059&view=rev
Log:
HIVE-1272. Add SymlinkTextInputFormat to Hive. (Guanghao Shen via zshao)

Added:
    hadoop/hive/trunk/data/files/symlink1.txt
    hadoop/hive/trunk/data/files/symlink2.txt
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java
    hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/symlink_text_input_format.q
    hadoop/hive/trunk/ql/src/test/results/clientpositive/symlink_text_input_format.q.out
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=930059&r1=930058&r2=930059&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Thu Apr  1 18:24:14 2010
@@ -63,6 +63,9 @@ Trunk -  Unreleased
     HIVE-1219. Robust handling of metastore connection failures
     (Paul Yang via namit)
 
+    HIVE-1272. Add SymlinkTextInputFormat to Hive.
+    (Guanghao Shen via zshao)
+
   IMPROVEMENTS
     HIVE-983. Function from_unixtime takes long.
     (Ning Zhang via zshao)

Added: hadoop/hive/trunk/data/files/symlink1.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/data/files/symlink1.txt?rev=930059&view=auto
==============================================================================
--- hadoop/hive/trunk/data/files/symlink1.txt (added)
+++ hadoop/hive/trunk/data/files/symlink1.txt Thu Apr  1 18:24:14 2010
@@ -0,0 +1,2 @@
+../data/files/T1.txt
+../data/files/T3.txt

Added: hadoop/hive/trunk/data/files/symlink2.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/data/files/symlink2.txt?rev=930059&view=auto
==============================================================================
--- hadoop/hive/trunk/data/files/symlink2.txt (added)
+++ hadoop/hive/trunk/data/files/symlink2.txt Thu Apr  1 18:24:14 2010
@@ -0,0 +1 @@
+../data/files/T2.txt

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java?rev=930059&r1=930058&r2=930059&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java Thu Apr  1 18:24:14 2010
@@ -29,6 +29,8 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -39,15 +41,13 @@ import org.apache.hadoop.hive.shims.Hado
 import org.apache.hadoop.hive.shims.HadoopShims.InputSplitShim;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 
 
 /**
@@ -224,8 +224,9 @@ public class CombineHiveInputFormat<K ex
     CombineFileInputFormatShim combine = ShimLoader.getHadoopShims()
         .getCombineFileInputFormat();
 
-    if (combine == null)
+    if (combine == null) {
       return super.getSplits(job, numSplits);
+    }
 
     if (combine.getInputPathsShim(job).length == 0) {
       throw new IOException("No input paths specified in job");
@@ -271,6 +272,10 @@ public class CombineHiveInputFormat<K ex
         return super.getSplits(job, numSplits);
       }
 
+      if (inputFormat instanceof SymlinkTextInputFormat) {
+        return super.getSplits(job, numSplits);
+      }
+
       combine.createPool(job, new CombineFilter(path));
     }
 

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java?rev=930059&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java Thu Apr  1 18:24:14 2010
@@ -0,0 +1,184 @@
+package org.apache.hadoop.hive.ql.io;
+
+import java.io.BufferedReader;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobConfigurable;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+/**
+ * Symlink file is a text file which contains a list of filename / dirname.
+ * This input method reads symlink files from specified job input paths and
+ * takes the files / directories specified in those symlink files as
+ * actual map-reduce input. The target input data should be in TextInputFormat.
+ */
+@SuppressWarnings("deprecation")
+public class SymlinkTextInputFormat
+    implements InputFormat<LongWritable, Text>, JobConfigurable {
+  /**
+   * This input split wraps the FileSplit generated from
+   * TextInputFormat.getSplits(), while setting the original link file path
+   * as job input path. This is needed because MapOperator relies on the
+   * job input path to lookup correct child operators. The target data file
+   * is encapsulated in the wrapped FileSplit.
+   */
+  public static class SymlinkTextInputSplit extends FileSplit {
+    private final FileSplit split;
+
+    public SymlinkTextInputSplit() {
+      super((Path)null, 0, 0, (String[])null);
+      split = new FileSplit((Path)null, 0, 0, (String[])null);
+    }
+
+    public SymlinkTextInputSplit(Path symlinkPath, FileSplit split) throws IOException {
+      super(symlinkPath, 0, 0, split.getLocations());
+      this.split = split;
+    }
+
+    /**
+     * Gets the target split, i.e. the split of target data.
+     */
+    public FileSplit getTargetSplit() {
+      return split;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      super.write(out);
+      split.write(out);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      super.readFields(in);
+      split.readFields(in);
+    }
+  }
+
+  @Override
+  public RecordReader<LongWritable, Text> getRecordReader(
+      InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    InputSplit targetSplit = ((SymlinkTextInputSplit)split).getTargetSplit();
+
+    // The target data is in TextInputFormat.
+    TextInputFormat inputFormat = new TextInputFormat();
+    inputFormat.configure(job);
+    return inputFormat.getRecordReader(targetSplit, job, reporter);
+  }
+
+  /**
+   * Parses all target paths from job input directory which contains symlink
+   * files, and splits the target data using TextInputFormat.
+   */
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits)
+      throws IOException {
+    Path[] symlinksDirs = FileInputFormat.getInputPaths(job);
+    if (symlinksDirs.length == 0) {
+      throw new IOException("No input paths specified in job.");
+    }
+
+    // Get all target paths first, because the number of total target paths
+    // is used to determine number of splits of each target path.
+    List<Path> targetPaths = new ArrayList<Path>();
+    List<Path> symlinkPaths = new ArrayList<Path>();
+    try {
+      getTargetPathsFromSymlinksDirs(
+          job,
+          symlinksDirs,
+          targetPaths,
+          symlinkPaths);
+    } catch (Exception e) {
+      throw new IOException(
+          "Error parsing symlinks from specified job input path.", e);
+    }
+    if (targetPaths.size() == 0) {
+      return new InputSplit[0];
+    }
+
+    // The input should be in TextInputFormat.
+    TextInputFormat inputFormat = new TextInputFormat();
+    JobConf newjob = new JobConf(job);
+    newjob.setInputFormat(TextInputFormat.class);
+    inputFormat.configure(newjob);
+
+    List<InputSplit> result = new ArrayList<InputSplit>();
+
+    // ceil(numSplits / numPaths), so we can get at least numSplits splits.
+    int numPaths = targetPaths.size();
+    int numSubSplits = (numSplits + numPaths - 1) / numPaths;
+
+    // For each path, do getSplits().
+    for (int i = 0; i < numPaths; ++i) {
+      Path targetPath = targetPaths.get(i);
+      Path symlinkPath = symlinkPaths.get(i);
+
+      FileInputFormat.setInputPaths(newjob, targetPath);
+
+      InputSplit[] iss = inputFormat.getSplits(newjob, numSubSplits);
+      for (InputSplit is : iss) {
+        result.add(new SymlinkTextInputSplit(symlinkPath, (FileSplit)is));
+      }
+    }
+    return result.toArray(new InputSplit[result.size()]);
+  }
+
+  @Override
+  public void configure(JobConf job) {
+    // empty
+  }
+
+  /**
+   * Given list of directories containing symlink files, read all target
+   * paths from symlink files and return as targetPaths list. And for each
+   * targetPaths[i], symlinkPaths[i] will be the path to the symlink file
+   * containing the target path.
+   */
+  private static void getTargetPathsFromSymlinksDirs(
+      Configuration conf, Path[] symlinksDirs,
+      List<Path> targetPaths, List<Path> symlinkPaths) throws IOException {
+    for (Path symlinkDir : symlinksDirs) {
+      FileSystem fileSystem = symlinkDir.getFileSystem(conf);
+      FileStatus[] symlinks = fileSystem.listStatus(symlinkDir);
+
+      // Read paths from each symlink file.
+      for (FileStatus symlink : symlinks) {
+        BufferedReader reader =
+            new BufferedReader(
+                new InputStreamReader(
+                    fileSystem.open(symlink.getPath())));
+
+        String line;
+        while ((line = reader.readLine()) != null) {
+          targetPaths.add(new Path(line));
+          symlinkPaths.add(symlink.getPath());
+        }
+      }
+    }
+  }
+
+  /**
+   * For backward compatibility with hadoop 0.17.
+   */
+  public void validateInput(JobConf job) throws IOException {
+    // do nothing
+  }
+}

Added: hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java?rev=930059&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java (added)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java Thu Apr  1 18:24:14 2010
@@ -0,0 +1,196 @@
+package org.apache.hadoop.hive.ql.io;
+
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Unittest for SymlinkTextInputFormat.
+ */
+@SuppressWarnings("deprecation")
+public class TestSymlinkTextInputFormat extends TestCase {
+  private static Log log =
+      LogFactory.getLog(TestSymlinkTextInputFormat.class);
+
+  private Configuration conf;
+  private JobConf job;
+  private FileSystem fileSystem;
+  private Path testDir;
+  Reporter reporter;
+
+  private Path dataDir1;
+  private Path dataDir2;
+  private Path symlinkDir;
+
+  @Override
+  protected void setUp() throws IOException {
+    conf = new Configuration();
+    job = new JobConf(conf);
+    fileSystem = FileSystem.getLocal(conf);
+    testDir = new Path(System.getProperty("test.data.dir", ".") +
+                       "/TestSymlinkTextInputFormat");
+    reporter = Reporter.NULL;
+    fileSystem.delete(testDir, true);
+
+    dataDir1 = new Path(testDir, "datadir1");
+    dataDir2 = new Path(testDir, "datadir2");
+    symlinkDir = new Path(testDir, "symlinkdir");
+  }
+
+  @Override
+  protected void tearDown() throws IOException {
+    fileSystem.delete(testDir, true);
+  }
+
+  /**
+   * Test scenario: Two data directories, one symlink file that contains two
+   * paths each point to a file in one of data directories.
+   */
+  public void testAccuracy1() throws IOException {
+    // First data dir, contains 2 files.
+    writeTextFile(new Path(dataDir1, "file1"),
+                  "dir1_file1_line1\n" +
+                  "dir1_file1_line2\n");
+    writeTextFile(new Path(dataDir1, "file2"),
+                  "dir1_file2_line1\n" +
+                  "dir1_file2_line2\n");
+
+    // Second data dir, contains 2 files.
+    writeTextFile(new Path(dataDir2, "file1"),
+                  "dir2_file1_line1\n" +
+                  "dir2_file1_line2\n");
+    writeTextFile(new Path(dataDir2, "file2"),
+                  "dir2_file2_line1\n" +
+                  "dir2_file2_line2\n");
+
+    // A symlink file, contains first file from first dir and second file from
+    // second dir.
+    writeSymlinkFile(
+        new Path(symlinkDir, "symlink_file"),
+        new Path(dataDir1, "file1"),
+        new Path(dataDir2, "file2"));
+
+    FileInputFormat.setInputPaths(job, symlinkDir);
+
+    SymlinkTextInputFormat inputFormat = new SymlinkTextInputFormat();
+    InputSplit[] splits = inputFormat.getSplits(job, 2);
+
+    log.info("Number of splits: " + splits.length);
+
+    // Read all values.
+    List<String> received = new ArrayList<String>();
+    for (InputSplit split : splits) {
+      RecordReader<LongWritable, Text> reader =
+          inputFormat.getRecordReader(split, job, reporter);
+
+      LongWritable key = reader.createKey();
+      Text value = reader.createValue();
+      while (reader.next(key, value)) {
+        received.add(value.toString());
+      }
+    }
+
+    List<String> expected = new ArrayList<String>();
+    expected.add("dir1_file1_line1");
+    expected.add("dir1_file1_line2");
+    expected.add("dir2_file2_line1");
+    expected.add("dir2_file2_line2");
+
+    assertEquals(expected, received);
+  }
+
+  /**
+   * Scenario: Empty input directory, i.e. no symlink file.
+   *
+   * Expected: Should return empty result set without any exception.
+   */
+  public void testAccuracy2() throws IOException {
+    fileSystem.mkdirs(symlinkDir);
+
+    FileInputFormat.setInputPaths(job, symlinkDir);
+
+    SymlinkTextInputFormat inputFormat = new SymlinkTextInputFormat();
+    InputSplit[] splits = inputFormat.getSplits(job, 2);
+
+    log.info("Number of splits: " + splits.length);
+
+    // Read all values.
+    List<String> received = new ArrayList<String>();
+    for (InputSplit split : splits) {
+      RecordReader<LongWritable, Text> reader =
+          inputFormat.getRecordReader(split, job, reporter);
+
+      LongWritable key = reader.createKey();
+      Text value = reader.createValue();
+      while (reader.next(key, value)) {
+        received.add(value.toString());
+      }
+    }
+
+    List<String> expected = new ArrayList<String>();
+
+    assertEquals(expected, received);
+  }
+
+  /**
+   * Scenario: No job input paths.
+   * Expected: IOException with proper message.
+   */
+  public void testFailure() {
+    SymlinkTextInputFormat inputFormat = new SymlinkTextInputFormat();
+
+    try {
+      inputFormat.getSplits(job, 2);
+      fail("IOException expected if no job input paths specified.");
+    } catch (IOException e) {
+      assertEquals("Incorrect exception message for no job input paths error.",
+                   "No input paths specified in job.",
+                   e.getMessage());
+    }
+  }
+
+  /**
+   * Writes the given string to the given file.
+   */
+  private void writeTextFile(Path file, String content) throws IOException {
+    OutputStreamWriter writer = new OutputStreamWriter(fileSystem.create(file));
+    writer.write(content);
+    writer.close();
+  }
+
+  /**
+   * Writes a symlink file that contains given list of paths.
+   *
+   * @param symlinkFile
+   * The symlink file to write.
+   *
+   * @param paths
+   * The list of paths to write to the symlink file.
+   */
+  private void writeSymlinkFile(Path symlinkFile, Path...paths)
+      throws IOException {
+    OutputStreamWriter writer =
+        new OutputStreamWriter(fileSystem.create(symlinkFile));
+    for (Path path : paths) {
+      writer.write(path.toString());
+      writer.write("\n");
+    }
+    writer.close();
+  }
+}

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/symlink_text_input_format.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/symlink_text_input_format.q?rev=930059&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/symlink_text_input_format.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/symlink_text_input_format.q Thu Apr  1 18:24:14 2010
@@ -0,0 +1,24 @@
+DROP TABLE symlink_text_input_format;
+
+EXPLAIN
+CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat';
+
+CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat';
+
+dfs -cp ../data/files/symlink1.txt ../build/ql/test/data/warehouse/symlink_text_input_format/symlink1.txt;
+dfs -cp ../data/files/symlink2.txt ../build/ql/test/data/warehouse/symlink_text_input_format/symlink2.txt;
+
+EXPLAIN SELECT * FROM symlink_text_input_format;
+
+SELECT * FROM symlink_text_input_format;
+
+EXPLAIN SELECT value FROM symlink_text_input_format;
+
+SELECT value FROM symlink_text_input_format;
+
+EXPLAIN SELECT count(1) FROM symlink_text_input_format;
+
+SELECT count(1) FROM symlink_text_input_format;
+
+DROP TABLE symlink_text_input_format;
+

Added: hadoop/hive/trunk/ql/src/test/results/clientpositive/symlink_text_input_format.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/symlink_text_input_format.q.out?rev=930059&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/symlink_text_input_format.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/symlink_text_input_format.q.out Thu Apr  1 18:24:14 2010
@@ -0,0 +1,202 @@
+PREHOOK: query: DROP TABLE symlink_text_input_format
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE symlink_text_input_format
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: EXPLAIN
+CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: EXPLAIN
+CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'
+POSTHOOK: type: CREATETABLE
+ABSTRACT SYNTAX TREE:
+  (TOK_CREATETABLE symlink_text_input_format TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL key TOK_STRING) (TOK_TABCOL value TOK_STRING)) (TOK_TABLEFILEFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' '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 string, value string
+          if not exists: false
+          input format: org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat
+          # buckets: -1
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          name: symlink_text_input_format
+          isExternal: false
+
+
+PREHOOK: query: CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@symlink_text_input_format
+PREHOOK: query: EXPLAIN SELECT * FROM symlink_text_input_format
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT * FROM symlink_text_input_format
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF symlink_text_input_format)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: SELECT * FROM symlink_text_input_format
+PREHOOK: type: QUERY
+PREHOOK: Input: default@symlink_text_input_format
+PREHOOK: Output: file:/data/users/guanghao/w1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-03-31_16-09-55_412_372868623647124211/10000
+POSTHOOK: query: SELECT * FROM symlink_text_input_format
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@symlink_text_input_format
+POSTHOOK: Output: file:/data/users/guanghao/w1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-03-31_16-09-55_412_372868623647124211/10000
+1	11
+2	12
+3	13
+7	17
+8	18
+8	28
+2	12
+4	14
+6	16
+7	17
+2	22
+3	13
+4	14
+5	15
+8	18
+8	18
+PREHOOK: query: EXPLAIN SELECT value FROM symlink_text_input_format
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT value FROM symlink_text_input_format
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF symlink_text_input_format)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL value)))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        symlink_text_input_format 
+          TableScan
+            alias: symlink_text_input_format
+            Select Operator
+              expressions:
+                    expr: value
+                    type: string
+              outputColumnNames: _col0
+              File Output Operator
+                compressed: false
+                GlobalTableId: 0
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: SELECT value FROM symlink_text_input_format
+PREHOOK: type: QUERY
+PREHOOK: Input: default@symlink_text_input_format
+PREHOOK: Output: file:/data/users/guanghao/w1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-03-31_16-09-55_939_1571446349980486687/10000
+POSTHOOK: query: SELECT value FROM symlink_text_input_format
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@symlink_text_input_format
+POSTHOOK: Output: file:/data/users/guanghao/w1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-03-31_16-09-55_939_1571446349980486687/10000
+11
+12
+13
+17
+18
+28
+12
+14
+16
+17
+22
+13
+14
+15
+18
+18
+PREHOOK: query: EXPLAIN SELECT count(1) FROM symlink_text_input_format
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT count(1) FROM symlink_text_input_format
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF symlink_text_input_format)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION count 1)))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        symlink_text_input_format 
+          TableScan
+            alias: symlink_text_input_format
+            Select Operator
+              Group By Operator
+                aggregations:
+                      expr: count(1)
+                bucketGroup: false
+                mode: hash
+                outputColumnNames: _col0
+                Reduce Output Operator
+                  sort order: 
+                  tag: -1
+                  value expressions:
+                        expr: _col0
+                        type: bigint
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(VALUE._col0)
+          bucketGroup: false
+          mode: mergepartial
+          outputColumnNames: _col0
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: bigint
+            outputColumnNames: _col0
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: SELECT count(1) FROM symlink_text_input_format
+PREHOOK: type: QUERY
+PREHOOK: Input: default@symlink_text_input_format
+PREHOOK: Output: file:/data/users/guanghao/w1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-03-31_16-09-59_427_4904352844938482139/10000
+POSTHOOK: query: SELECT count(1) FROM symlink_text_input_format
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@symlink_text_input_format
+POSTHOOK: Output: file:/data/users/guanghao/w1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-03-31_16-09-59_427_4904352844938482139/10000
+16
+PREHOOK: query: DROP TABLE symlink_text_input_format
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE symlink_text_input_format
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: default@symlink_text_input_format