Przeglądaj źródła

HADOOP-4753. Refactor gridmix2 to reduce code duplication.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.20@727145 13f79535-47bb-0310-9956-ffa450edef68
Christopher Douglas 16 lat temu
rodzic
commit
defeeb021d

+ 2 - 0
CHANGES.txt

@@ -282,6 +282,8 @@ Release 0.20.0 - Unreleased
     HADOOP-4545. Add example and test case of secondary sort for the reduce.
     (omalley)
 
+    HADOOP-4753. Refactor gridmix2 to reduce code duplication. (cdouglas)
+
   OPTIMIZATIONS
 
     HADOOP-3293. Fixes FileInputFormat to do provide locations for splits

+ 1 - 1
src/benchmarks/gridmix2/src/java/org/apache/hadoop/mapred/CombinerJobCreator.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.io.Text;
 
 public class CombinerJobCreator extends WordCount {
 
-  public JobConf createJob(String[] args) throws Exception {
+  public static JobConf createJob(String[] args) throws Exception {
     JobConf conf = new JobConf(WordCount.class);
     conf.setJobName("GridmixCombinerJob");
 

+ 1 - 1
src/benchmarks/gridmix2/src/java/org/apache/hadoop/mapred/GenericMRLoadJobCreator.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.mapred.JobConf;
 
 public class GenericMRLoadJobCreator extends GenericMRLoadGenerator {
 
-  public JobConf createJob(String[] argv, boolean mapoutputCompressed,
+  public static JobConf createJob(String[] argv, boolean mapoutputCompressed,
       boolean outputCompressed) throws Exception {
 
     JobConf job = new JobConf();

+ 0 - 34
src/benchmarks/gridmix2/src/java/org/apache/hadoop/mapred/GridMixConfig.java

@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapred;
-
-import org.apache.hadoop.conf.Configuration;
-
-public class GridMixConfig extends Configuration {
-
-  public int[] getInts(String name, int defautValue) {
-    String[] valuesInString = getStrings(name, String.valueOf(defautValue));
-    int[] results = new int[valuesInString.length];
-    for (int i = 0; i < valuesInString.length; i++) {
-      results[i] = Integer.parseInt(valuesInString[i]);
-    }
-    return results;
-
-  }
-}

Plik diff jest za duży
+ 338 - 1222
src/benchmarks/gridmix2/src/java/org/apache/hadoop/mapred/GridMixRunner.java


+ 0 - 121
src/benchmarks/gridmix2/src/java/org/apache/hadoop/mapred/SortJobCreator.java

@@ -1,121 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapred;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.examples.Sort;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.lib.IdentityMapper;
-import org.apache.hadoop.mapred.lib.IdentityReducer;
-import org.apache.hadoop.conf.Configured;
-
-public class SortJobCreator extends Configured {
-
-  public JobConf createJob(String[] args) throws Exception {
-
-    // JobConf jobConf = new JobConf(getConf(), Sort.class);
-    JobConf jobConf = new JobConf();
-    jobConf.setJarByClass(Sort.class);
-    jobConf.setJobName("GridmixJavaSorter");
-
-    jobConf.setMapperClass(IdentityMapper.class);
-    jobConf.setReducerClass(IdentityReducer.class);
-
-    JobClient client = new JobClient(jobConf);
-    ClusterStatus cluster = client.getClusterStatus();
-    int num_reduces = (int) (cluster.getMaxReduceTasks() * 0.9);
-    String sort_reduces = jobConf.get("test.sort.reduces_per_host");
-    if (sort_reduces != null) {
-      num_reduces = cluster.getTaskTrackers() * Integer.parseInt(sort_reduces);
-    }
-    Class<? extends InputFormat> inputFormatClass = SequenceFileInputFormat.class;
-    Class<? extends OutputFormat> outputFormatClass = SequenceFileOutputFormat.class;
-    Class<? extends WritableComparable> outputKeyClass = BytesWritable.class;
-    Class<? extends Writable> outputValueClass = BytesWritable.class;
-    boolean mapoutputCompressed = false;
-    boolean outputCompressed = false;
-    List<String> otherArgs = new ArrayList<String>();
-    for (int i = 0; i < args.length; ++i) {
-      try {
-        if ("-m".equals(args[i])) {
-          jobConf.setNumMapTasks(Integer.parseInt(args[++i]));
-
-        } else if ("-r".equals(args[i])) {
-          num_reduces = Integer.parseInt(args[++i]);
-        } else if ("-inFormat".equals(args[i])) {
-          inputFormatClass = Class.forName(args[++i]).asSubclass(
-              InputFormat.class);
-        } else if ("-outFormat".equals(args[i])) {
-          outputFormatClass = Class.forName(args[++i]).asSubclass(
-              OutputFormat.class);
-        } else if ("-outKey".equals(args[i])) {
-          outputKeyClass = Class.forName(args[++i]).asSubclass(
-              WritableComparable.class);
-        } else if ("-outValue".equals(args[i])) {
-          outputValueClass = Class.forName(args[++i])
-              .asSubclass(Writable.class);
-        } else if ("-mapoutputCompressed".equals(args[i])) {
-          mapoutputCompressed = Boolean.valueOf(args[++i]).booleanValue();
-        } else if ("-outputCompressed".equals(args[i])) {
-          outputCompressed = Boolean.valueOf(args[++i]).booleanValue();
-        } else {
-          otherArgs.add(args[i]);
-        }
-      } catch (NumberFormatException except) {
-        System.out.println("ERROR: Integer expected instead of " + args[i]);
-        return null;
-      } catch (ArrayIndexOutOfBoundsException except) {
-        System.out.println("ERROR: Required parameter missing from "
-            + args[i - 1]);
-        return null; // exits
-      }
-    }
-
-    // Set user-supplied (possibly default) job configs
-    jobConf.setNumReduceTasks(num_reduces);
-
-    jobConf.setInputFormat(inputFormatClass);
-    jobConf.setOutputFormat(outputFormatClass);
-
-    jobConf.setOutputKeyClass(outputKeyClass);
-    jobConf.setOutputValueClass(outputValueClass);
-    jobConf.setCompressMapOutput(mapoutputCompressed);
-    jobConf.setBoolean("mapred.output.compress", outputCompressed);
-
-    // Make sure there are exactly 2 parameters left.
-    if (otherArgs.size() != 2) {
-      System.out.println("ERROR: Wrong number of parameters: "
-          + otherArgs.size() + " instead of 2.");
-      return null;
-    } //jobConf.setInputPath(new Path(otherArgs.get(0)));
-
-    FileInputFormat.addInputPaths(jobConf, otherArgs.get(0));
-
-    FileOutputFormat.setOutputPath(jobConf, new Path(otherArgs.get(1)));
-
-    return jobConf;
-  }
-
-}

+ 1 - 1
src/test/org/apache/hadoop/mapred/GenericMRLoadGenerator.java

@@ -61,7 +61,7 @@ public class GenericMRLoadGenerator extends Configured implements Tool {
   /**
    * Configure a job given argv.
    */
-  public boolean parseArgs(String[] argv, JobConf job) throws IOException {
+  public static boolean parseArgs(String[] argv, JobConf job) throws IOException {
     if (argv.length < 1) {
       return 0 == printUsage();
     }

Niektóre pliki nie zostały wyświetlone z powodu dużej ilości zmienionych plików