pig-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From da...@apache.org
Subject svn commit: r1564564 - in /pig/trunk/src/org/apache/pig: ./ backend/hadoop/executionengine/mapReduceLayer/ builtin/ newplan/logical/rules/
Date Tue, 04 Feb 2014 23:41:42 GMT
Author: daijy
Date: Tue Feb  4 23:41:41 2014
New Revision: 1564564

URL: http://svn.apache.org/r1564564
Log:
PIG-259: allow store to overwrite existing directroy (PIG-259.9.patch)

Removed:
    pig/trunk/src/org/apache/pig/OverwritingStoreFunc.java
Modified:
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java
    pig/trunk/src/org/apache/pig/builtin/PigStorage.java
    pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java?rev=1564564&r1=1564563&r2=1564564&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
(original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
Tue Feb  4 23:41:41 2014
@@ -60,7 +60,7 @@ import org.apache.hadoop.mapred.jobcontr
 import org.apache.pig.ComparisonFunc;
 import org.apache.pig.ExecType;
 import org.apache.pig.LoadFunc;
-import org.apache.pig.OverwritingStoreFunc;
+import org.apache.pig.OverwritableStoreFunc;
 import org.apache.pig.PigConfiguration;
 import org.apache.pig.PigException;
 import org.apache.pig.StoreFuncInterface;
@@ -669,9 +669,9 @@ public class JobControlCompiler{
                 storeLocations.add(st);
                 StoreFuncInterface sFunc = st.getStoreFunc();
                 sFunc.setStoreLocation(st.getSFile().getFileName(), nwJob);
-                if (sFunc instanceof OverwritingStoreFunc) {
-                    OverwritingStoreFunc osf = (OverwritingStoreFunc) sFunc;
-                    if (osf.isOverwrite()) {
+                if (sFunc instanceof OverwritableStoreFunc) {
+                    OverwritableStoreFunc osf = (OverwritableStoreFunc) sFunc;
+                    if (osf.shouldOverwrite()) {
                         osf.cleanupOutput(st, nwJob);
                     }
                 }
@@ -681,9 +681,9 @@ public class JobControlCompiler{
                 storeLocations.add(st);
                 StoreFuncInterface sFunc = st.getStoreFunc();
                 sFunc.setStoreLocation(st.getSFile().getFileName(), nwJob);
-                if (sFunc instanceof OverwritingStoreFunc) {
-                    OverwritingStoreFunc osf = (OverwritingStoreFunc) sFunc;
-                    if (osf.isOverwrite()) {
+                if (sFunc instanceof OverwritableStoreFunc) {
+                    OverwritableStoreFunc osf = (OverwritableStoreFunc) sFunc;
+                    if (osf.shouldOverwrite()) {
                         osf.cleanupOutput(st, nwJob);
                     }
                 }

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java?rev=1564564&r1=1564563&r2=1564564&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java
(original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java
Tue Feb  4 23:41:41 2014
@@ -31,7 +31,7 @@ import org.apache.hadoop.mapreduce.Outpu
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.pig.OverwritingStoreFunc;
+import org.apache.pig.OverwritableStoreFunc;
 import org.apache.pig.StoreFuncInterface;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
@@ -212,8 +212,8 @@ public class PigOutputFormat extends Out
                 of.checkOutputSpecs(jobContextCopy);
             } catch (IOException ioe) {
                 boolean shouldThrowException = true;
-                if (sFunc instanceof OverwritingStoreFunc) {
-                    if (((OverwritingStoreFunc) sFunc).isOverwrite()) {
+                if (sFunc instanceof OverwritableStoreFunc) {
+                    if (((OverwritableStoreFunc) sFunc).shouldOverwrite()) {
                         if (ioe instanceof FileAlreadyExistsException
                                 || ioe instanceof org.apache.hadoop.fs.FileAlreadyExistsException)
{
                             shouldThrowException = false;

Modified: pig/trunk/src/org/apache/pig/builtin/PigStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/PigStorage.java?rev=1564564&r1=1564563&r2=1564564&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/PigStorage.java (original)
+++ pig/trunk/src/org/apache/pig/builtin/PigStorage.java Tue Feb  4 23:41:41 2014
@@ -54,7 +54,7 @@ import org.apache.pig.LoadCaster;
 import org.apache.pig.LoadFunc;
 import org.apache.pig.LoadMetadata;
 import org.apache.pig.LoadPushDown;
-import org.apache.pig.OverwritingStoreFunc;
+import org.apache.pig.OverwritableStoreFunc;
 import org.apache.pig.PigException;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.ResourceSchema.ResourceFieldSchema;
@@ -132,7 +132,7 @@ import org.apache.pig.parser.ParserExcep
  */
 @SuppressWarnings("unchecked")
 public class PigStorage extends FileInputLoadFunc implements StoreFuncInterface,
-LoadPushDown, LoadMetadata, StoreMetadata, OverwritingStoreFunc {
+LoadPushDown, LoadMetadata, StoreMetadata, OverwritableStoreFunc {
     protected RecordReader in = null;
     protected RecordWriter writer = null;
     protected final Log mLog = LogFactory.getLog(getClass());
@@ -586,7 +586,7 @@ LoadPushDown, LoadMetadata, StoreMetadat
     }
 
     @Override
-    public boolean isOverwrite() {
+    public boolean shouldOverwrite() {
         return this.overwriteOutput;
     }
 

Modified: pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java?rev=1564564&r1=1564563&r2=1564564&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java (original)
+++ pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java Tue Feb
 4 23:41:41 2014
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.pig.OverwritingStoreFunc;
+import org.apache.pig.OverwritableStoreFunc;
 import org.apache.pig.PigException;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.StoreFuncInterface;
@@ -95,8 +95,8 @@ public class InputOutputFileValidator {
                 }
                 
                 boolean shouldThrowException = true;
-                if (sf instanceof OverwritingStoreFunc) {
-                    if (((OverwritingStoreFunc) sf).isOverwrite()) {
+                if (sf instanceof OverwritableStoreFunc) {
+                    if (((OverwritableStoreFunc) sf).shouldOverwrite()) {
                         if (ioe instanceof FileAlreadyExistsException
                                 || ioe instanceof org.apache.hadoop.fs.FileAlreadyExistsException)
{
                             shouldThrowException = false;



Mime
View raw message