You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by pr...@apache.org on 2010/01/29 21:56:11 UTC
svn commit: r904619 - in /hadoop/pig/branches/load-store-redesign:
src/org/apache/pig/
src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
src/org/apache/pig/backend/hadoop/executionengine/util/
src/org/apache/pig/backend/hadoop/hbase/ te...
Author: pradeepkth
Date: Fri Jan 29 20:56:10 2010
New Revision: 904619
URL: http://svn.apache.org/viewvc?rev=904619&view=rev
Log:
PIG-1090: additional commit to remove StoreConfig and CommittableStoreFunc which are no longer needed (pradeepkth)
Removed:
hadoop/pig/branches/load-store-redesign/src/org/apache/pig/CommittableStoreFunc.java
hadoop/pig/branches/load-store-redesign/src/org/apache/pig/StoreConfig.java
Modified:
hadoop/pig/branches/load-store-redesign/src/org/apache/pig/SortColInfo.java
hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputCommitter.java
hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java
hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java
hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestMultiQuery.java
Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/SortColInfo.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/SortColInfo.java?rev=904619&r1=904618&r2=904619&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/SortColInfo.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/SortColInfo.java Fri Jan 29 20:56:10 2010
@@ -22,8 +22,7 @@
import org.apache.pig.impl.util.Utils;
/**
- * A class representing information about a sort column to pass
- * in {@link SortInfo} to storage functions in {@link StoreConfig}
+ * A class representing information about a sort column in {@link SortInfo}
*/
public class SortColInfo implements Serializable {
Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java?rev=904619&r1=904618&r2=904619&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java Fri Jan 29 20:56:10 2010
@@ -17,7 +17,6 @@
*/
package org.apache.pig.backend.hadoop.executionengine.mapReduceLayer;
-import java.io.DataInput;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
@@ -25,7 +24,6 @@
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.HashMap;
-import java.util.Hashtable;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
@@ -48,12 +46,11 @@
import org.apache.pig.FuncSpec;
import org.apache.pig.PigException;
import org.apache.pig.ResourceSchema;
-import org.apache.pig.StoreConfig;
import org.apache.pig.StoreFunc;
import org.apache.pig.backend.executionengine.ExecException;
import org.apache.pig.backend.hadoop.HDataType;
-import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.SkewedPartitioner;
import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.SecondaryKeyPartitioner;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.SkewedPartitioner;
import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner;
import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MROperPlan;
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
@@ -114,8 +111,6 @@
private final Log log = LogFactory.getLog(getClass());
- public static final String PIG_STORE_CONFIG = "pig.store.config";
-
public static final String LOG_DIR = "_logs";
public static final String END_OF_INP_IN_MAP = "pig.invoke.close.in.map";
@@ -484,10 +479,6 @@
// in the store funcspec would break the job.xml which is created by
// hadoop from the jobconf.
conf.set(PIG_STORE_FUNC, ObjectSerializer.serialize(outputFuncSpec.toString()));
- conf.set(PIG_STORE_CONFIG,
- ObjectSerializer.serialize(
- new StoreConfig(outputPath, st.getSchema(), st.getSortInfo())));
-
conf.set("pig.streaming.log.dir",
new Path(outputPath, LOG_DIR).toString());
conf.set("pig.streaming.task.output.dir", outputPath);
Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputCommitter.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputCommitter.java?rev=904619&r1=904618&r2=904619&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputCommitter.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputCommitter.java Fri Jan 29 20:56:10 2010
@@ -22,8 +22,6 @@
import java.util.LinkedList;
import java.util.List;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
@@ -132,7 +130,7 @@
// side effects like setting map.output.dir on the Configuration
// in the Context are needed by the OutputCommitter, those actions
// will be done before the committer is created. Also the String
- // version of StoreConfig and StoreFunc for the specific store need
+ // version of StoreFunc for the specific store need
// to be set up in the context in case the committer needs them
PigOutputFormat.setLocation(contextCopy, store);
return contextCopy;
@@ -149,7 +147,7 @@
// side effects like setting map.output.dir on the Configuration
// in the Context are needed by the OutputCommitter, those actions
// will be done before the committer is created. Also the String
- // version of StoreConfig and StoreFunc for the specific store need
+ // version of StoreFunc for the specific store need
// to be set up in the context in case the committer needs them
PigOutputFormat.setLocation(contextCopy, store);
return contextCopy;
Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java?rev=904619&r1=904618&r2=904619&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java Fri Jan 29 20:56:10 2010
@@ -19,8 +19,6 @@
import java.io.IOException;
import java.util.List;
-import java.util.ArrayList;
-import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.WritableComparable;
@@ -30,13 +28,9 @@
import org.apache.hadoop.mapreduce.OutputFormat;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.pig.StoreConfig;
import org.apache.pig.StoreFunc;
import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
import org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil;
import org.apache.pig.data.Tuple;
import org.apache.pig.impl.util.ObjectSerializer;
@@ -63,6 +57,7 @@
* place to store the output from a number of map-reduce tasks*/
public static final String PIG_TMP_PATH = "pig.tmp.path";
+ @Override
public RecordWriter<WritableComparable, Tuple> getRecordWriter(TaskAttemptContext taskattemptcontext)
throws IOException, InterruptedException {
// Setup UDFContext so StoreFunc can make use of it
@@ -159,7 +154,7 @@
/**
* Before delegating calls to underlying OutputFormat or OutputCommitter
* Pig needs to ensure the Configuration in the JobContext contains
- * the output location and the String version of StoreConfig and StoreFunc
+ * the output location and StoreFunc
* for the specific store - so set these up in the context for this specific
* store
* @param jobContext the {@link JobContext}
@@ -186,8 +181,8 @@
// Before delegating calls to underlying OutputFormat or OutputCommitter
// Pig needs to ensure the Configuration in the JobContext contains
- // the String version of StoreCoonfig and StoreFunc for the specific
- // store - so set these up in the context for this specific store
+ // StoreFunc for the specific store - so set this up in the context
+ // for this specific store
updateContextWithStoreInfo(jobContext, store);
}
@@ -249,7 +244,6 @@
/**
* Before delegating calls to underlying OutputFormat or OutputCommitter
* Pig needs to ensure the Configuration in the {@link JobContext} contains
- * {@link JobControlCompiler#PIG_STORE_CONFIG} and
* {@link JobControlCompiler#PIG_STORE_FUNC}. This helper method can be
* used to set this up
* @param context the job context
@@ -259,10 +253,6 @@
public static void updateContextWithStoreInfo(JobContext context,
POStore store) throws IOException {
Configuration conf = context.getConfiguration();
- conf.set(JobControlCompiler.PIG_STORE_CONFIG,
- ObjectSerializer.serialize(new StoreConfig(
- store.getSFile().getFileName(), store.getSchema(),
- store.getSortInfo())));
conf.set(JobControlCompiler.PIG_STORE_FUNC,
store.getSFile().getFuncSpec().toString());
Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java?rev=904619&r1=904618&r2=904619&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java Fri Jan 29 20:56:10 2010
@@ -19,7 +19,6 @@
package org.apache.pig.backend.hadoop.executionengine.util;
import java.io.IOException;
-import java.io.InputStream;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
@@ -28,21 +27,16 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.util.Progressable;
import org.apache.pig.PigException;
-import org.apache.pig.StoreConfig;
import org.apache.pig.StoreFunc;
import org.apache.pig.backend.executionengine.ExecException;
-import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
-import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler;
import org.apache.pig.builtin.BinStorage;
import org.apache.pig.builtin.PigStorage;
-import org.apache.pig.impl.io.ReadToEndLoader;
import org.apache.pig.data.DataBag;
import org.apache.pig.data.DataType;
import org.apache.pig.data.DefaultTupleFactory;
import org.apache.pig.data.Tuple;
import org.apache.pig.impl.PigContext;
-import org.apache.pig.impl.io.BufferedPositionedInputStream;
-import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.io.ReadToEndLoader;
import org.apache.pig.impl.util.ObjectSerializer;
import org.apache.pig.impl.util.Pair;
import org.apache.pig.impl.util.UDFContext;
@@ -54,8 +48,8 @@
/**
* This method is to be called from an
- * {@link org.apache.hadoop.mapred.OutputFormat#getRecordWriter(FileSystem ignored, JobConf job,
- String name, Progressable progress)}
+ * {@link org.apache.hadoop.mapred.OutputFormat#getRecordWriter(
+ * FileSystem, org.apache.hadoop.mapred.JobConf, String, Progressable)}
* method to obtain a reference to the {@link org.apache.pig.StoreFunc} object to be used by
* that OutputFormat to perform the write() operation
* @param conf the JobConf object
@@ -82,22 +76,6 @@
}
/**
- * This method is to be called from an
- * {@link org.apache.hadoop.mapred.OutputFormat#getRecordWriter(FileSystem ignored, JobConf job,
- String name, Progressable progress)}
- * method to obtain a reference to the {@link org.apache.pig.StoreConfig} object. The StoreConfig
- * object will contain metadata information like schema and location to be used by
- * that OutputFormat to perform the write() operation
- * @param conf the JobConf object
- * @return StoreConfig object containing metadata information useful for
- * an OutputFormat to write the data
- * @throws IOException
- */
- public static StoreConfig getStoreConfig(Configuration conf) throws IOException {
- return (StoreConfig) ObjectSerializer.deserialize(conf.get(JobControlCompiler.PIG_STORE_CONFIG));
- }
-
- /**
* Loads the key distribution sampler file
*
* @param keyDistFile the name for the distribution file
Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java?rev=904619&r1=904618&r2=904619&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java Fri Jan 29 20:56:10 2010
@@ -42,9 +42,7 @@
import org.apache.pig.data.TupleFactory;
/**
- * A <code>Slicer</code> that split the hbase table into {@link HBaseSlice}s.
- * And a load function will provided to do none load operations, the actually
- * load operatrions will be done in {@link HBaseSlice}.
+ * A Hbase Loader
*/
public class HBaseStorage extends LoadFunc {
@@ -56,9 +54,6 @@
private static final Log LOG = LogFactory.getLog(HBaseStorage.class);
- // HBase Slicer
- // Creates a slice per region of a specified table.
-
/**
* Constructor. Construct a HBase Table loader to load the cells of the
* provided columns.
Modified: hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestMultiQuery.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestMultiQuery.java?rev=904619&r1=904618&r2=904619&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestMultiQuery.java (original)
+++ hadoop/pig/branches/load-store-redesign/test/org/apache/pig/test/TestMultiQuery.java Fri Jan 29 20:56:10 2010
@@ -17,13 +17,14 @@
*/
package org.apache.pig.test;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
-import java.io.OutputStream;
import java.io.PrintWriter;
import java.io.StringReader;
import java.util.ArrayList;
@@ -34,25 +35,21 @@
import java.util.Map;
import junit.framework.Assert;
-import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.RecordWriter;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.OutputFormat;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.util.Progressable;
import org.apache.pig.ExecType;
import org.apache.pig.LoadFunc;
import org.apache.pig.PigException;
import org.apache.pig.PigServer;
import org.apache.pig.ResourceSchema;
-import org.apache.pig.StoreConfig;
import org.apache.pig.StoreFunc;
import org.apache.pig.backend.executionengine.ExecJob;
import org.apache.pig.backend.executionengine.util.ExecTools;
@@ -63,7 +60,6 @@
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit;
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
-import org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil;
import org.apache.pig.data.DataBag;
import org.apache.pig.data.Tuple;
import org.apache.pig.impl.PigContext;
@@ -2696,7 +2692,8 @@
@Override
public void setStoreLocation(String location, Job job)
throws IOException {
- // TODO Auto-generated method stub
+ Configuration conf = job.getConfiguration();
+ conf.set("mapred.output.dir", location);
}
@@ -2716,11 +2713,10 @@
@Override
public void checkOutputSpecs(JobContext context) throws IOException,
InterruptedException {
- StoreConfig sConfig = MapRedUtil.getStoreConfig(context.
- getConfiguration());
- FileSystem fs = FileSystem.get(context.getConfiguration());
+ Configuration conf = context.getConfiguration();
+ FileSystem fs = FileSystem.get(conf);
// create a file to test that this method got called
- fs.create(new Path(sConfig.getLocation() + "_checkOutputSpec_test"));
+ fs.create(new Path(conf.get("mapred.output.dir") + "_checkOutputSpec_test"));
}
@Override