You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2013/10/23 17:13:20 UTC

svn commit: r1535051 - in /hive/trunk/hcatalog: core/src/main/java/org/apache/hive/hcatalog/data/transfer/impl/ core/src/main/java/org/apache/hive/hcatalog/mapreduce/ core/src/test/java/org/apache/hive/hcatalog/mapreduce/ hcatalog-pig-adapter/src/main/...

Author: brock
Date: Wed Oct 23 15:13:20 2013
New Revision: 1535051

URL: http://svn.apache.org/r1535051
Log:
HIVE-5454 - HCatalog runs a partition listing with an empty filter (Harsh J via Brock Noland)

Modified:
    hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/transfer/impl/HCatInputFormatReader.java
    hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatInputFormat.java
    hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatMapReduceTest.java
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java
    hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/inputoutput.xml
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/GroupByAge.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadJson.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadRC.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadText.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadWrite.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SimpleRead.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreComplex.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreDemo.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreNumbers.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SumNumbers.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/TypeDataCheck.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteJson.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteRC.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteText.java
    hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hive/hcatalog/hbase/TestHBaseInputFormat.java

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/transfer/impl/HCatInputFormatReader.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/transfer/impl/HCatInputFormatReader.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/transfer/impl/HCatInputFormatReader.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/transfer/impl/HCatInputFormatReader.java Wed Oct 23 15:13:20 2013
@@ -63,7 +63,7 @@ public class HCatInputFormatReader exten
     try {
       Job job = new Job(conf);
       HCatInputFormat hcif = HCatInputFormat.setInput(
-        job, re.getDbName(), re.getTableName()).setFilter(re.getFilterString());
+        job, re.getDbName(), re.getTableName(), re.getFilterString());
       ReaderContext cntxt = new ReaderContext();
       cntxt.setInputSplits(hcif.getSplits(
           ShimLoader.getHadoopShims().getHCatShim().createJobContext(job.getConfiguration(), null)));

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatInputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatInputFormat.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatInputFormat.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatInputFormat.java Wed Oct 23 15:13:20 2013
@@ -39,28 +39,33 @@ public class HCatInputFormat extends HCa
   private InputJobInfo inputJobInfo;
 
   /**
-   * @deprecated as of release 0.5, and will be removed in a future release
+   * Initializes the input with a null filter.
+   * See {@link #setInput(org.apache.hadoop.conf.Configuration, String, String, String)}
    */
-  @Deprecated
-  public static void setInput(Job job, InputJobInfo inputJobInfo) throws IOException {
-    setInput(job.getConfiguration(), inputJobInfo);
+  public static HCatInputFormat setInput(
+          Job job, String dbName, String tableName)
+    throws IOException {
+    return setInput(job.getConfiguration(), dbName, tableName, null);
   }
 
   /**
-   * @deprecated as of release 0.5, and will be removed in a future release
+   * Initializes the input with a provided filter.
+   * See {@link #setInput(org.apache.hadoop.conf.Configuration, String, String, String)}
    */
-  @Deprecated
-  public static void setInput(Configuration conf, InputJobInfo inputJobInfo) throws IOException {
-    setInput(conf, inputJobInfo.getDatabaseName(), inputJobInfo.getTableName())
-      .setFilter(inputJobInfo.getFilter())
-      .setProperties(inputJobInfo.getProperties());
+  public static HCatInputFormat setInput(
+          Job job, String dbName, String tableName, String filter)
+    throws IOException {
+    return setInput(job.getConfiguration(), dbName, tableName, filter);
   }
 
   /**
-   * See {@link #setInput(org.apache.hadoop.conf.Configuration, String, String)}
+   * Initializes the input with a null filter.
+   * See {@link #setInput(org.apache.hadoop.conf.Configuration, String, String, String)}
    */
-  public static HCatInputFormat setInput(Job job, String dbName, String tableName) throws IOException {
-    return setInput(job.getConfiguration(), dbName, tableName);
+  public static HCatInputFormat setInput(
+          Configuration conf, String dbName, String tableName)
+    throws IOException {
+    return setInput(conf, dbName, tableName, null);
   }
 
   /**
@@ -69,9 +74,11 @@ public class HCatInputFormat extends HCa
    * @param conf the job configuration
    * @param dbName database name, which if null 'default' is used
    * @param tableName table name
+   * @param filter the partition filter to use, can be null for no filter
    * @throws IOException on all errors
    */
-  public static HCatInputFormat setInput(Configuration conf, String dbName, String tableName)
+  public static HCatInputFormat setInput(
+          Configuration conf, String dbName, String tableName, String filter)
     throws IOException {
 
     Preconditions.checkNotNull(conf, "required argument 'conf' is null");
@@ -79,7 +86,7 @@ public class HCatInputFormat extends HCa
 
     HCatInputFormat hCatInputFormat = new HCatInputFormat();
     hCatInputFormat.conf = conf;
-    hCatInputFormat.inputJobInfo = InputJobInfo.create(dbName, tableName, null, null);
+    hCatInputFormat.inputJobInfo = InputJobInfo.create(dbName, tableName, filter, null);
 
     try {
       InitializeInput.setInput(conf, hCatInputFormat.inputJobInfo);
@@ -91,11 +98,11 @@ public class HCatInputFormat extends HCa
   }
 
   /**
-   * Set a filter on the input table.
-   * @param filter the filter specification, which may be null
-   * @return this
-   * @throws IOException on all errors
+   * @deprecated As of 0.13
+   * Use {@link #setInput(org.apache.hadoop.conf.Configuration, String, String, String)} instead,
+   * to specify a partition filter to directly initialize the input with.
    */
+  @Deprecated
   public HCatInputFormat setFilter(String filter) throws IOException {
     // null filters are supported to simplify client code
     if (filter != null) {

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatMapReduceTest.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatMapReduceTest.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatMapReduceTest.java (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatMapReduceTest.java Wed Oct 23 15:13:20 2013
@@ -341,7 +341,7 @@ public abstract class HCatMapReduceTest 
     job.setInputFormatClass(HCatInputFormat.class);
     job.setOutputFormatClass(TextOutputFormat.class);
 
-    HCatInputFormat.setInput(job, dbName, tableName).setFilter(filter);
+    HCatInputFormat.setInput(job, dbName, tableName, filter);
 
     job.setMapOutputKeyClass(BytesWritable.class);
     job.setMapOutputValueClass(Text.class);

Modified: hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java (original)
+++ hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java Wed Oct 23 15:13:20 2013
@@ -116,7 +116,7 @@ public class HCatLoader extends HCatBase
       }
     } else {
       Job clone = new Job(job.getConfiguration());
-      HCatInputFormat.setInput(job, dbName, tableName).setFilter(getPartitionFilterString());
+      HCatInputFormat.setInput(job, dbName, tableName, getPartitionFilterString());
 
       // We will store all the new /changed properties in the job in the
       // udf context, so the the HCatInputFormat.setInput method need not

Modified: hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/inputoutput.xml
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/inputoutput.xml?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/inputoutput.xml (original)
+++ hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/inputoutput.xml Wed Oct 23 15:13:20 2013
@@ -45,9 +45,7 @@
 	    <li><code>getTableSchema</code></li>
 	</ul>
 
-	<p>To use HCatInputFormat to read data, first instantiate an <code>InputJobInfo</code>
-	with the necessary information from the table being read
-	and then call setInput with the <code>InputJobInfo</code>.</p>
+	<p>To use HCatInputFormat to read data, call setInput with the database name, tablename and an optional partition filter.</p>
 
 <p>You can use the <code>setOutputSchema</code> method to include a projection schema, to
 specify the output fields. If a schema is not specified, all the columns in the table
@@ -62,11 +60,13 @@ will be returned.</p>
    * the information in the conf object. The inputInfo object is updated with
    * information needed in the client context
    * @param job the job object
-   * @param inputJobInfo the input info for table to read
+   * @param dbName the database where the table lies
+   * @param tableName the table to read
+   * @param filter the partition filter to use
    * @throws IOException the exception in communicating with the metadata server
    */
   public static void setInput(Job job,
-      InputJobInfo inputJobInfo) throws IOException;
+      String dbName, String tableName, String filter) throws IOException;
 
   /**
    * Set the schema for the HCatRecord data returned by HCatInputFormat.
@@ -354,8 +354,8 @@ public class GroupByAge extends Configur
         String dbName = null;
 
         Job job = new Job(conf, "GroupByAge");
-        HCatInputFormat.setInput(job, InputJobInfo.create(dbName,
-                inputTableName, null));
+        HCatInputFormat.setInput(job, dbName,
+                inputTableName);
         // initialize HCatOutputFormat
 
         job.setInputFormatClass(HCatInputFormat.class);
@@ -388,8 +388,8 @@ public class GroupByAge extends Configur
 <li>The implementation of Map takes HCatRecord as an input and the implementation of Reduce produces it as an output.</li>
 <li>This example program assumes the schema of the input, but it could also retrieve the schema via
 HCatOutputFormat.getOutputSchema() and retrieve fields based on the results of that call.</li>
-<li>The input descriptor for the table to be read is created by calling InputJobInfo.create.  It requires the database name,
-table name, and partition filter.  In this example the partition filter is null, so all partitions of the table
+<li>The input descriptor for the table to be read is created by passing the database name,
+table name, and an optional partition filter to HCatInputFormat.setInput. In this example the partition filter is null, so all partitions of the table
 will be read.</li>
 <li>The output descriptor for the table to be written is created by calling OutputJobInfo.create.  It requires the
 database name, the table name, and a Map of partition keys and values that describe the partition being written.
@@ -397,7 +397,7 @@ In this example it is assumed the table 
 </ol>
 
 <p>To scan just selected partitions of a table, a filter describing the desired partitions can be passed to
-InputJobInfo.create.  To scan a single partition, the filter string should look like: "<code>ds=20120401</code>"
+HCatInputFormat.setInput.  To scan a single partition, the filter string should look like: "<code>ds=20120401</code>"
 where the datestamp "<code>ds</code>" is the partition column name and "<code>20120401</code>" is the value
 you want to read (year, month, and day).</p>
 </section>
@@ -420,14 +420,14 @@ you want to read (year, month, and day).
 
 <p>Assume for example you have a web_logs table that is partitioned by the column "<code>ds</code>".  You could select one partition of the table by changing</p>
 <source>
-HCatInputFormat.setInput(job, InputJobInfo.create(dbName, inputTableName, null));
+HCatInputFormat.setInput(job, dbName, inputTableName);
 </source>
 <p>
 to
 </p>
 <source>
 HCatInputFormat.setInput(job,
-                         InputJobInfo.create(dbName, inputTableName, "ds=\"20110924\""));
+                         dbName, inputTableName, "ds=\"20110924\"");
 </source>
 <p>
 This filter must reference only partition columns.  Values from other columns will cause the job to fail.</p>

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java Wed Oct 23 15:13:20 2013
@@ -41,7 +41,6 @@ import org.apache.hcatalog.data.DefaultH
 import org.apache.hcatalog.data.HCatRecord;
 import org.apache.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -165,8 +164,7 @@ public class HBaseReadWrite extends Conf
     if (!succ) return 1;
 
     job = new Job(conf, "HBaseRead");
-    HCatInputFormat.setInput(job, InputJobInfo.create(dbName, tableName,
-      null));
+    HCatInputFormat.setInput(job, dbName, tableName);
 
     job.setInputFormatClass(HCatInputFormat.class);
     job.setOutputFormatClass(TextOutputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/GroupByAge.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/GroupByAge.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/GroupByAge.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/GroupByAge.java Wed Oct 23 15:13:20 2013
@@ -38,7 +38,6 @@ import org.apache.hive.hcatalog.data.HCa
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -105,8 +104,8 @@ public class GroupByAge extends Configur
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "GroupByAge");
-    HCatInputFormat.setInput(job, InputJobInfo.create(dbName,
-      inputTableName, null));
+    HCatInputFormat.setInput(job, dbName,
+      inputTableName);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadJson.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadJson.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadJson.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadJson.java Wed Oct 23 15:13:20 2013
@@ -37,7 +37,6 @@ import org.apache.hive.hcatalog.common.H
 import org.apache.hive.hcatalog.data.DefaultHCatRecord;
 import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 
 /**
  * This is a map reduce test for testing hcat which goes against the "numbers"
@@ -90,8 +89,8 @@ public class ReadJson extends Configured
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "ReadJson");
-    HCatInputFormat.setInput(job, InputJobInfo.create(
-      dbName, tableName, null));
+    HCatInputFormat.setInput(job,
+      dbName, tableName);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadRC.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadRC.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadRC.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadRC.java Wed Oct 23 15:13:20 2013
@@ -37,7 +37,6 @@ import org.apache.hive.hcatalog.common.H
 import org.apache.hive.hcatalog.data.DefaultHCatRecord;
 import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 
 /**
  * This is a map reduce test for testing hcat which goes against the "numbers"
@@ -91,8 +90,8 @@ public class ReadRC extends Configured i
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "ReadRC");
-    HCatInputFormat.setInput(job, InputJobInfo.create(
-      dbName, tableName, null));
+    HCatInputFormat.setInput(job,
+      dbName, tableName);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadText.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadText.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadText.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadText.java Wed Oct 23 15:13:20 2013
@@ -37,7 +37,6 @@ import org.apache.hive.hcatalog.common.H
 import org.apache.hive.hcatalog.data.DefaultHCatRecord;
 import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 
 /**
  * This is a map reduce test for testing hcat which goes against the "numbers"
@@ -102,8 +101,8 @@ public class ReadText extends Configured
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "ReadText");
-    HCatInputFormat.setInput(job, InputJobInfo.create(
-      dbName, tableName, null));
+    HCatInputFormat.setInput(job,
+      dbName, tableName);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadWrite.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadWrite.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadWrite.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/ReadWrite.java Wed Oct 23 15:13:20 2013
@@ -36,7 +36,6 @@ import org.apache.hive.hcatalog.data.HCa
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -86,8 +85,8 @@ public class ReadWrite extends Configure
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "ReadWrite");
-    HCatInputFormat.setInput(job, InputJobInfo.create(dbName,
-      inputTableName, null));
+    HCatInputFormat.setInput(job, dbName,
+      inputTableName);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SimpleRead.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SimpleRead.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SimpleRead.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SimpleRead.java Wed Oct 23 15:13:20 2013
@@ -37,7 +37,6 @@ import org.apache.hadoop.util.ToolRunner
 import org.apache.hive.hcatalog.common.HCatConstants;
 import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 
 /**
  * This is a map reduce test for testing hcat which goes against the "numbers"
@@ -87,8 +86,8 @@ public class SimpleRead extends Configur
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "SimpleRead");
-    HCatInputFormat.setInput(job, InputJobInfo.create(
-      dbName, tableName, null));
+    HCatInputFormat.setInput(job,
+      dbName, tableName, null);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreComplex.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreComplex.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreComplex.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreComplex.java Wed Oct 23 15:13:20 2013
@@ -36,7 +36,6 @@ import org.apache.hive.hcatalog.data.HCa
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -103,8 +102,8 @@ public class StoreComplex {
     Job job = new Job(conf, "storecomplex");
     // initialize HCatInputFormat
 
-    HCatInputFormat.setInput(job, InputJobInfo.create(
-      dbName, tableName, null));
+    HCatInputFormat.setInput(job,
+      dbName, tableName);
     // initialize HCatOutputFormat
     HCatOutputFormat.setOutput(job, OutputJobInfo.create(
       dbName, outputTableName, outputPartitionKvps));

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreDemo.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreDemo.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreDemo.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreDemo.java Wed Oct 23 15:13:20 2013
@@ -35,7 +35,6 @@ import org.apache.hive.hcatalog.data.HCa
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -114,8 +113,8 @@ public class StoreDemo {
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "storedemo");
     // initialize HCatInputFormat
-    HCatInputFormat.setInput(job, InputJobInfo.create(
-      dbName, tableName, null));
+    HCatInputFormat.setInput(job,
+      dbName, tableName);
     // initialize HCatOutputFormat
     HCatOutputFormat.setOutput(job, OutputJobInfo.create(
       dbName, outputTableName, outputPartitionKvps));

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreNumbers.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreNumbers.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreNumbers.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/StoreNumbers.java Wed Oct 23 15:13:20 2013
@@ -39,7 +39,6 @@ import org.apache.hive.hcatalog.data.sch
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -178,8 +177,8 @@ public class StoreNumbers {
     Job job = new Job(conf, "storenumbers");
 
     // initialize HCatInputFormat
-    HCatInputFormat.setInput(job, InputJobInfo.create(
-      dbName, tableName, null));
+    HCatInputFormat.setInput(job,
+      dbName, tableName);
     // initialize HCatOutputFormat
     HCatOutputFormat.setOutput(job, OutputJobInfo.create(
       dbName, outputTableName, outputPartitionKvps));

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SumNumbers.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SumNumbers.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SumNumbers.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/SumNumbers.java Wed Oct 23 15:13:20 2013
@@ -41,7 +41,6 @@ import org.apache.hadoop.util.GenericOpt
 import org.apache.hive.hcatalog.common.HCatConstants;
 import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 
 /**
  * This is a map reduce test for testing hcat which goes against the "numbers"
@@ -162,8 +161,8 @@ public class SumNumbers {
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "sumnumbers");
-    HCatInputFormat.setInput(job, InputJobInfo.create(
-      dbName, tableName, null));
+    HCatInputFormat.setInput(job,
+      dbName, tableName);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/TypeDataCheck.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/TypeDataCheck.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/TypeDataCheck.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/TypeDataCheck.java Wed Oct 23 15:13:20 2013
@@ -37,7 +37,6 @@ import org.apache.hive.hcatalog.common.H
 import org.apache.hive.hcatalog.data.HCatRecord;
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 
 /**
  * This is a map reduce test for testing hcat that checks that the columns
@@ -150,8 +149,8 @@ public class TypeDataCheck implements To
       }
       Job job = new Job(conf, "typedatacheck");
       // initialize HCatInputFormat
-      HCatInputFormat.setInput(job, InputJobInfo.create(
-        dbName, tableName, null));
+      HCatInputFormat.setInput(job,
+        dbName, tableName);
       HCatSchema s = HCatInputFormat.getTableSchema(job);
       job.getConfiguration().set(SCHEMA_KEY, schemaStr);
       job.getConfiguration().set(DELIM, outputdelim);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteJson.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteJson.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteJson.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteJson.java Wed Oct 23 15:13:20 2013
@@ -35,7 +35,6 @@ import org.apache.hive.hcatalog.data.HCa
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -91,8 +90,8 @@ public class WriteJson extends Configure
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "WriteJson");
-    HCatInputFormat.setInput(job, InputJobInfo.create(dbName,
-      inputTableName, null));
+    HCatInputFormat.setInput(job, dbName,
+      inputTableName);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteRC.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteRC.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteRC.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteRC.java Wed Oct 23 15:13:20 2013
@@ -35,7 +35,6 @@ import org.apache.hive.hcatalog.data.HCa
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -93,8 +92,8 @@ public class WriteRC extends Configured 
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "WriteRC");
-    HCatInputFormat.setInput(job, InputJobInfo.create(dbName,
-      inputTableName, null));
+    HCatInputFormat.setInput(job, dbName,
+      inputTableName);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteText.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteText.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteText.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteText.java Wed Oct 23 15:13:20 2013
@@ -35,7 +35,6 @@ import org.apache.hive.hcatalog.data.HCa
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -103,8 +102,8 @@ public class WriteText extends Configure
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "WriteText");
-    HCatInputFormat.setInput(job, InputJobInfo.create(dbName,
-      inputTableName, null));
+    HCatInputFormat.setInput(job, dbName,
+      inputTableName);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java (original)
+++ hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java Wed Oct 23 15:13:20 2013
@@ -39,7 +39,6 @@ import org.apache.hive.hcatalog.data.sch
 import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
 
 /**
@@ -94,8 +93,8 @@ public class WriteTextPartitioned extend
     if (principalID != null)
       conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
     Job job = new Job(conf, "WriteTextPartitioned");
-    HCatInputFormat.setInput(job, InputJobInfo.create(dbName,
-      inputTableName, filter));
+    HCatInputFormat.setInput(job, dbName,
+      inputTableName, filter);
     // initialize HCatOutputFormat
 
     job.setInputFormatClass(HCatInputFormat.class);

Modified: hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hive/hcatalog/hbase/TestHBaseInputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hive/hcatalog/hbase/TestHBaseInputFormat.java?rev=1535051&r1=1535050&r2=1535051&view=diff
==============================================================================
--- hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hive/hcatalog/hbase/TestHBaseInputFormat.java (original)
+++ hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hive/hcatalog/hbase/TestHBaseInputFormat.java Wed Oct 23 15:13:20 2013
@@ -62,7 +62,6 @@ import org.apache.hive.hcatalog.data.HCa
 import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
 import org.junit.Test;
 
 public class TestHBaseInputFormat extends SkeletonHBaseTest {
@@ -160,9 +159,7 @@ public class TestHBaseInputFormat extend
     MapReadHTable.resetCounters();
 
     job.setInputFormatClass(HCatInputFormat.class);
-    InputJobInfo inputJobInfo = InputJobInfo.create(databaseName, tableName,
-                null);
-    HCatInputFormat.setInput(job, inputJobInfo);
+    HCatInputFormat.setInput(job, databaseName, tableName);
     job.setOutputFormatClass(TextOutputFormat.class);
     TextOutputFormat.setOutputPath(job, outputDir);
     job.setMapOutputKeyClass(BytesWritable.class);
@@ -225,10 +222,9 @@ public class TestHBaseInputFormat extend
     job.setJarByClass(this.getClass());
     job.setMapperClass(MapReadProjHTable.class);
     job.setInputFormatClass(HCatInputFormat.class);
-    InputJobInfo inputJobInfo = InputJobInfo.create(
-      MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, null);
     HCatInputFormat.setOutputSchema(job, getProjectionSchema());
-    HCatInputFormat.setInput(job, inputJobInfo);
+    HCatInputFormat.setInput(job,
+      MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
     job.setOutputFormatClass(TextOutputFormat.class);
     TextOutputFormat.setOutputPath(job, outputDir);
     job.setMapOutputKeyClass(BytesWritable.class);