flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From rmetz...@apache.org
Subject [4/4] git commit: Hadoop Compat pull request clean-up
Date Sat, 21 Jun 2014 09:45:28 GMT
Hadoop Compat pull request clean-up

This closes #32.


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/ff69b1bb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/ff69b1bb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/ff69b1bb

Branch: refs/heads/master
Commit: ff69b1bb2fd049d89f4fbd8c6946240a150043f6
Parents: a65b759
Author: Robert Metzger <rmetzger@apache.org>
Authored: Sat Jun 21 11:43:13 2014 +0200
Committer: Robert Metzger <rmetzger@apache.org>
Committed: Sat Jun 21 11:44:51 2014 +0200

----------------------------------------------------------------------
 .../mapred/HadoopInputFormat.java                  | 11 +++++------
 .../mapred/HadoopOutputFormat.java                 | 11 +++++------
 .../datatypes/HadoopFileOutputCommitter.java       |  3 +--
 .../mapred/utils/HadoopUtils.java                  |  6 ++----
 .../mapred/wrapper/HadoopInputSplit.java           |  6 +++---
 .../mapreduce/HadoopInputFormat.java               | 17 ++++++-----------
 .../mapreduce/HadoopOutputFormat.java              |  4 +---
 .../mapreduce/utils/HadoopUtils.java               |  6 ++----
 .../mapreduce/wrapper/HadoopInputSplit.java        |  3 +--
 9 files changed, 26 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ff69b1bb/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopInputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopInputFormat.java
b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopInputFormat.java
index 882f4a3..eddee69 100644
--- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopInputFormat.java
+++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopInputFormat.java
@@ -53,10 +53,10 @@ public class HadoopInputFormat<K extends Writable, V extends Writable>
implement
 	private Class<V> valueClass;
 	private JobConf jobConf;
 	
-	public transient K key;
-	public transient V value;
+	private transient K key;
+	private transient V value;
 	
-	public RecordReader<K, V> recordReader;
+	private transient RecordReader<K, V> recordReader;
 	private transient boolean fetched = false;
 	private transient boolean hasNext;
 	
@@ -117,10 +117,9 @@ public class HadoopInputFormat<K extends Writable, V extends Writable>
implement
 				LOG.warn("Could not determine statistics due to an io error: "
 						+ ioex.getMessage());
 			}
-		}
-		catch (Throwable t) {
+		} catch (Throwable t) {
 			if (LOG.isErrorEnabled()) {
-				LOG.error("Unexpected problen while getting the file statistics: "
+				LOG.error("Unexpected problem while getting the file statistics: "
 						+ t.getMessage(), t);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ff69b1bb/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopOutputFormat.java
b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopOutputFormat.java
index 849c701..d0b94ab 100644
--- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopOutputFormat.java
+++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/HadoopOutputFormat.java
@@ -39,10 +39,10 @@ public class HadoopOutputFormat<K extends Writable,V extends Writable>
implement
 	
 	private static final long serialVersionUID = 1L;
 	
-	public JobConf jobConf;	
-	public org.apache.hadoop.mapred.OutputFormat<K,V> mapredOutputFormat;	
-	public transient RecordWriter<K,V> recordWriter;	
-	public transient FileOutputCommitter fileOutputCommitter;
+	private JobConf jobConf;	
+	private org.apache.hadoop.mapred.OutputFormat<K,V> mapredOutputFormat;	
+	private transient RecordWriter<K,V> recordWriter;	
+	private transient FileOutputCommitter fileOutputCommitter;
 	private transient TaskAttemptContext context;
 	private transient JobContext jobContext;
 	
@@ -109,8 +109,7 @@ public class HadoopOutputFormat<K extends Writable,V extends Writable>
implement
 		
 		try {
 			this.jobContext = HadoopUtils.instantiateJobContext(this.jobConf, new JobID());
-		}
-		catch (Exception e) {
+		} catch (Exception e) {
 			throw new RuntimeException(e);
 		}
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ff69b1bb/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/record/datatypes/HadoopFileOutputCommitter.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/record/datatypes/HadoopFileOutputCommitter.java
b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/record/datatypes/HadoopFileOutputCommitter.java
index 8f46c00..aa848b3 100644
--- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/record/datatypes/HadoopFileOutputCommitter.java
+++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/record/datatypes/HadoopFileOutputCommitter.java
@@ -50,8 +50,7 @@ public class HadoopFileOutputCommitter extends FileOutputCommitter implements
Se
 	}
 
 	private static boolean getOutputDirMarking(JobConf conf) {
-		return conf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,
-			true);
+		return conf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,  true);
 	}
 
 	private void markSuccessfulOutputDir(JobConf conf)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ff69b1bb/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/utils/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/utils/HadoopUtils.java
b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/utils/HadoopUtils.java
index 18b30da..4886e4a 100644
--- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/utils/HadoopUtils.java
+++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/utils/HadoopUtils.java
@@ -54,8 +54,7 @@ public class HadoopUtils {
 			JobContext context = (JobContext) constructor.newInstance(jobConf, jobId);
 			
 			return context;
-		}
-		catch(Exception e) {
+		} catch(Exception e) {
 			throw new Exception("Could not create instance of JobContext.", e);
 		}
 	}
@@ -76,8 +75,7 @@ public class HadoopUtils {
 			constructor.setAccessible(true);
 			TaskAttemptContext context = (TaskAttemptContext) constructor.newInstance(jobConf, taskAttemptID);
 			return context;
-		}
-		catch(Exception e) {
+		} catch(Exception e) {
 			throw new Exception("Could not create instance of TaskAttemptContext.", e);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ff69b1bb/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java
b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java
index 5eb185d..980d0e6 100644
--- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java
+++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java
@@ -25,8 +25,8 @@ import eu.stratosphere.core.io.InputSplit;
 
 public class HadoopInputSplit implements InputSplit {
 
-	public transient org.apache.hadoop.mapred.InputSplit hadoopInputSplit;
-	public JobConf jobConf;
+	private transient org.apache.hadoop.mapred.InputSplit hadoopInputSplit;
+	private JobConf jobConf;
 	private int splitNumber;
 	private String hadoopInputSplitTypeName;
 	
@@ -44,7 +44,7 @@ public class HadoopInputSplit implements InputSplit {
 	public HadoopInputSplit(org.apache.hadoop.mapred.InputSplit hInputSplit, JobConf jobconf)
{
 		this.hadoopInputSplit = hInputSplit;
 		this.hadoopInputSplitTypeName = hInputSplit.getClass().getName();
-		this.jobConf=jobconf;
+		this.jobConf = jobconf;
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ff69b1bb/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopInputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopInputFormat.java
b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopInputFormat.java
index 2e847bd..455ca66 100644
--- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopInputFormat.java
+++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopInputFormat.java
@@ -56,7 +56,7 @@ public class HadoopInputFormat<K extends Writable, V extends Writable>
implement
 	private Class<V> valueClass;
 	private org.apache.hadoop.conf.Configuration configuration;
 	
-	public transient RecordReader<K, V> recordReader;
+	private transient RecordReader<K, V> recordReader;
 	private boolean fetched = false;
 	private boolean hasNext;
 	
@@ -108,8 +108,7 @@ public class HadoopInputFormat<K extends Writable, V extends Writable>
implement
 		JobContext jobContext = null;
 		try {
 			jobContext = HadoopUtils.instantiateJobContext(configuration, null);
-		}
-		catch (Exception e) {
+		} catch (Exception e) {
 			throw new RuntimeException(e);
 		}
 		
@@ -118,17 +117,15 @@ public class HadoopInputFormat<K extends Writable, V extends Writable>
implement
 				
 				try {
 					final org.apache.hadoop.fs.Path[] paths = FileInputFormat.getInputPaths(jobContext);
-					
 					return getFileStats(cachedFileStats, paths, new ArrayList<FileStatus>(1));
 				} catch (IOException ioex) {
 					if (LOG.isWarnEnabled()) {
 						LOG.warn("Could not determine statistics due to an io error: "
 								+ ioex.getMessage());
 					}
-				}
-				catch (Throwable t) {
+				} catch (Throwable t) {
 					if (LOG.isErrorEnabled()) {
-						LOG.error("Unexpected problen while getting the file statistics: "
+						LOG.error("Unexpected problem while getting the file statistics: "
 								+ t.getMessage(), t);
 					}
 				}
@@ -145,8 +142,7 @@ public class HadoopInputFormat<K extends Writable, V extends Writable>
implement
 		JobContext jobContext = null;
 		try {
 			jobContext = HadoopUtils.instantiateJobContext(configuration, new JobID());
-		}
-		catch (Exception e) {
+		} catch (Exception e) {
 			throw new RuntimeException(e);
 		}
 		
@@ -174,8 +170,7 @@ public class HadoopInputFormat<K extends Writable, V extends Writable>
implement
 		TaskAttemptContext context = null;
 		try {
 			context = HadoopUtils.instantiateTaskAttemptContext(configuration, new TaskAttemptID());
-		}
-		catch(Exception e) {
+		} catch(Exception e) {
 			throw new RuntimeException(e);
 		}
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ff69b1bb/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopOutputFormat.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopOutputFormat.java
b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopOutputFormat.java
index 2ec82de..f2e48c2 100644
--- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopOutputFormat.java
+++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/HadoopOutputFormat.java
@@ -111,8 +111,7 @@ public class HadoopOutputFormat<K extends Writable,V extends Writable>
implement
 		
 		try {
 			this.fileOutputCommitter.setupJob(HadoopUtils.instantiateJobContext(this.configuration,
new JobID()));
-		}
-		catch (Exception e) {
+		} catch (Exception e) {
 			throw new RuntimeException(e);
 		}
 		
@@ -140,7 +139,6 @@ public class HadoopOutputFormat<K extends Writable,V extends Writable>
implement
 	 * commit the task by moving the output file out from the temporary directory.
 	 * @throws IOException
 	 */
-	@SuppressWarnings("deprecation")
 	@Override
 	public void close() throws IOException {
 		try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ff69b1bb/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/utils/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/utils/HadoopUtils.java
b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/utils/HadoopUtils.java
index e46aa37..5b0b632 100644
--- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/utils/HadoopUtils.java
+++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/utils/HadoopUtils.java
@@ -51,8 +51,7 @@ public class HadoopUtils {
 			JobContext context = (JobContext) constructor.newInstance(configuration, jobId);
 			
 			return context;
-		}
-		catch(Exception e) {
+		} catch(Exception e) {
 			throw new Exception("Could not create instance of JobContext.");
 		}
 	}
@@ -72,8 +71,7 @@ public class HadoopUtils {
 			TaskAttemptContext context = (TaskAttemptContext) constructor.newInstance(configuration,
taskAttemptID);
 			
 			return context;
-		}
-		catch(Exception e) {
+		} catch(Exception e) {
 			throw new Exception("Could not create instance of TaskAttemptContext.");
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ff69b1bb/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java
b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java
index 656339b..985ad59 100644
--- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java
+++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java
@@ -67,8 +67,7 @@ public class HadoopInputSplit implements InputSplit {
 				Class<? extends org.apache.hadoop.io.Writable> inputSplit = 
 						Class.forName(className).asSubclass(org.apache.hadoop.io.Writable.class);
 				this.mapreduceInputSplit = (org.apache.hadoop.mapreduce.InputSplit) WritableFactories.newInstance(inputSplit);
-			}
-			catch (Exception e) {
+			} catch (Exception e) {
 				throw new RuntimeException("Unable to create InputSplit", e);
 			}
 		}


Mime
View raw message