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);