瀏覽代碼

HDFS-13149. Ozone: Rename Corona to Freon. Contributed by Anu Engineer.

Anu Engineer 7 年之前
父節點
當前提交
fc84744f75

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -38,7 +38,6 @@ function hadoop_usage
   hadoop_add_subcommand "cblock" admin "cblock CLI"
   hadoop_add_subcommand "cblockserver" daemon "run cblock server"
   hadoop_add_subcommand "classpath" client "prints the class path needed to get the hadoop jar and the required libraries"
-  hadoop_add_subcommand "corona" client "run an ozone data generator"
   hadoop_add_subcommand "crypto" admin "configure HDFS encryption zones"
   hadoop_add_subcommand "datanode" daemon "run a DFS datanode"
   hadoop_add_subcommand "debug" admin "run a Debug Admin to execute HDFS debug commands"
@@ -50,6 +49,7 @@ function hadoop_usage
   hadoop_add_subcommand "envvars" client "display computed Hadoop environment variables"
   hadoop_add_subcommand "ec" admin "run a HDFS ErasureCoding CLI"
   hadoop_add_subcommand "fetchdt" client "fetch a delegation token from the NameNode"
+  hadoop_add_subcommand "freon" client "runs an ozone data generator"
   hadoop_add_subcommand "fsck" admin "run a DFS filesystem checking utility"
   hadoop_add_subcommand "getconf" client "get config values from configuration"
   hadoop_add_subcommand "groups" client "get the groups which users belong to"
@@ -107,9 +107,6 @@ function hdfscmd_case
     classpath)
       hadoop_do_classpath_subcommand HADOOP_CLASSNAME "$@"
     ;;
-    corona)
-      HADOOP_CLASSNAME=org.apache.hadoop.ozone.tools.Corona
-    ;;
     crypto)
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.CryptoAdmin
     ;;
@@ -160,6 +157,9 @@ function hdfscmd_case
     fetchdt)
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.DelegationTokenFetcher
     ;;
+    freon)
+      HADOOP_CLASSNAME=org.apache.hadoop.ozone.tools.Freon
+    ;;
     fsck)
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.DFSck
     ;;

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -979,8 +979,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     ReplicaInfo newReplicaInfo = targetVolume.moveBlockToTmpLocation(block,
         replicaInfo, smallBufferSize, conf);
 
-     // Latch here --> wait for the signal.
-
     // Finalize the copied files
     newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
     try (AutoCloseableLock lock = datasetLock.acquire()) {

+ 57 - 57
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/tools/Corona.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/tools/Freon.java

@@ -62,7 +62,7 @@ import java.util.function.Supplier;
 import static java.lang.Math.min;
 
 /**
- * Corona - A tool to populate ozone with data for testing.<br>
+ * Freon - A tool to populate ozone with data for testing.<br>
  * This is not a map-reduce program and this is not for benchmarking
  * Ozone write throughput.<br>
  * It supports both online and offline modes. Default mode is offline,
@@ -92,9 +92,9 @@ import static java.lang.Math.min;
  * can be used to override</li>
  * </ul>
  */
-public final class Corona extends Configured implements Tool {
+public final class Freon extends Configured implements Tool {
 
-  enum CoronaOps {
+  enum FreonOps {
     VOLUME_CREATE,
     BUCKET_CREATE,
     KEY_CREATE,
@@ -127,7 +127,7 @@ public final class Corona extends Configured implements Tool {
   private static final int QUANTILES = 10;
 
   private static final Logger LOG =
-      LoggerFactory.getLogger(Corona.class);
+      LoggerFactory.getLogger(Freon.class);
 
   private boolean printUsage = false;
   private boolean completed = false;
@@ -176,7 +176,7 @@ public final class Corona extends Configured implements Tool {
   private ArrayList<Histogram> histograms = new ArrayList<>();
 
   @VisibleForTesting
-  Corona(Configuration conf) throws IOException {
+  Freon(Configuration conf) throws IOException {
     startTime = System.nanoTime();
     jobStartTime = System.currentTimeMillis();
     volumeCreationTime = new AtomicLong();
@@ -189,7 +189,7 @@ public final class Corona extends Configured implements Tool {
     numberOfKeysAdded = new AtomicLong();
     ozoneClient = OzoneClientFactory.getClient(conf);
     objectStore = ozoneClient.getObjectStore();
-    for (CoronaOps ops : CoronaOps.values()) {
+    for (FreonOps ops : FreonOps.values()) {
       histograms.add(ops.ordinal(), new Histogram(new UniformReservoir()));
     }
   }
@@ -199,7 +199,7 @@ public final class Corona extends Configured implements Tool {
    */
   public static void main(String[] args) throws Exception {
     Configuration conf = new OzoneConfiguration();
-    int res = ToolRunner.run(conf, new Corona(conf), args);
+    int res = ToolRunner.run(conf, new Freon(conf), args);
     System.exit(res);
   }
 
@@ -271,7 +271,7 @@ public final class Corona extends Configured implements Tool {
     OptionBuilder.withArgName("online | offline");
     OptionBuilder.hasArg();
     OptionBuilder.withDescription("specifies the mode of " +
-        "Corona run.");
+        "Freon run.");
     Option optMode = OptionBuilder.create(MODE);
 
     OptionBuilder.withArgName("source url");
@@ -400,7 +400,7 @@ public final class Corona extends Configured implements Tool {
     System.out.println("-jsonDir                        "
         + "directory where json is created.");
     System.out.println("-mode [online | offline]        "
-        + "specifies the mode in which Corona should run.");
+        + "specifies the mode in which Freon should run.");
     System.out.println("-source <url>                   "
         + "specifies the URL of s3 commoncrawl warc file to " +
         "be used when the mode is online.");
@@ -446,7 +446,7 @@ public final class Corona extends Configured implements Tool {
   }
 
   /**
-   * Prints stats of {@link Corona} run to the PrintStream.
+   * Prints stats of {@link Freon} run to the PrintStream.
    *
    * @param out PrintStream
    */
@@ -512,10 +512,10 @@ public final class Corona extends Configured implements Tool {
     if (jsonDir != null) {
 
       String[][] quantileTime =
-          new String[CoronaOps.values().length][QUANTILES + 1];
-      String[] deviations = new String[CoronaOps.values().length];
-      String[] means = new String[CoronaOps.values().length];
-      for (CoronaOps ops : CoronaOps.values()) {
+          new String[FreonOps.values().length][QUANTILES + 1];
+      String[] deviations = new String[FreonOps.values().length];
+      String[] means = new String[FreonOps.values().length];
+      for (FreonOps ops : FreonOps.values()) {
         Snapshot snapshot = histograms.get(ops.ordinal()).getSnapshot();
         for (int i = 0; i <= QUANTILES; i++) {
           quantileTime[ops.ordinal()][i] = DurationFormatUtils.formatDuration(
@@ -531,26 +531,26 @@ public final class Corona extends Configured implements Tool {
             DURATION_FORMAT);
       }
 
-      CoronaJobInfo jobInfo = new CoronaJobInfo().setExecTime(execTime)
+      FreonJobInfo jobInfo = new FreonJobInfo().setExecTime(execTime)
           .setGitBaseRevision(VersionInfo.getRevision())
-          .setMeanVolumeCreateTime(means[CoronaOps.VOLUME_CREATE.ordinal()])
+          .setMeanVolumeCreateTime(means[FreonOps.VOLUME_CREATE.ordinal()])
           .setDeviationVolumeCreateTime(
-              deviations[CoronaOps.VOLUME_CREATE.ordinal()])
+              deviations[FreonOps.VOLUME_CREATE.ordinal()])
           .setTenQuantileVolumeCreateTime(
-              quantileTime[CoronaOps.VOLUME_CREATE.ordinal()])
-          .setMeanBucketCreateTime(means[CoronaOps.BUCKET_CREATE.ordinal()])
+              quantileTime[FreonOps.VOLUME_CREATE.ordinal()])
+          .setMeanBucketCreateTime(means[FreonOps.BUCKET_CREATE.ordinal()])
           .setDeviationBucketCreateTime(
-              deviations[CoronaOps.BUCKET_CREATE.ordinal()])
+              deviations[FreonOps.BUCKET_CREATE.ordinal()])
           .setTenQuantileBucketCreateTime(
-              quantileTime[CoronaOps.BUCKET_CREATE.ordinal()])
-          .setMeanKeyCreateTime(means[CoronaOps.KEY_CREATE.ordinal()])
-          .setDeviationKeyCreateTime(deviations[CoronaOps.KEY_CREATE.ordinal()])
+              quantileTime[FreonOps.BUCKET_CREATE.ordinal()])
+          .setMeanKeyCreateTime(means[FreonOps.KEY_CREATE.ordinal()])
+          .setDeviationKeyCreateTime(deviations[FreonOps.KEY_CREATE.ordinal()])
           .setTenQuantileKeyCreateTime(
-              quantileTime[CoronaOps.KEY_CREATE.ordinal()])
-          .setMeanKeyWriteTime(means[CoronaOps.KEY_WRITE.ordinal()])
-          .setDeviationKeyWriteTime(deviations[CoronaOps.KEY_WRITE.ordinal()])
+              quantileTime[FreonOps.KEY_CREATE.ordinal()])
+          .setMeanKeyWriteTime(means[FreonOps.KEY_WRITE.ordinal()])
+          .setDeviationKeyWriteTime(deviations[FreonOps.KEY_WRITE.ordinal()])
           .setTenQuantileKeyWriteTime(
-              quantileTime[CoronaOps.KEY_WRITE.ordinal()]);
+              quantileTime[FreonOps.KEY_WRITE.ordinal()]);
       String jsonName =
           new SimpleDateFormat("yyyyMMddHHmmss").format(Time.now()) + ".json";
       String jsonPath = jsonDir + "/" + jsonName;
@@ -704,7 +704,7 @@ public final class Corona extends Configured implements Tool {
         objectStore.createVolume(volumeName);
         long volumeCreationDuration = System.nanoTime() - start;
         volumeCreationTime.getAndAdd(volumeCreationDuration);
-        histograms.get(CoronaOps.VOLUME_CREATE.ordinal())
+        histograms.get(FreonOps.VOLUME_CREATE.ordinal())
             .update(volumeCreationDuration);
         numberOfVolumesCreated.getAndIncrement();
         volume = objectStore.getVolume(volumeName);
@@ -724,7 +724,7 @@ public final class Corona extends Configured implements Tool {
           start = System.nanoTime();
           volume.createBucket(bucketName);
           long bucketCreationDuration = System.nanoTime() - start;
-          histograms.get(CoronaOps.BUCKET_CREATE.ordinal())
+          histograms.get(FreonOps.BUCKET_CREATE.ordinal())
               .update(bucketCreationDuration);
           bucketCreationTime.getAndAdd(bucketCreationDuration);
           numberOfBucketsCreated.getAndIncrement();
@@ -741,7 +741,7 @@ public final class Corona extends Configured implements Tool {
               OzoneOutputStream os =
                   bucket.createKey(key, keySize, type, factor);
               long keyCreationDuration = System.nanoTime() - keyCreateStart;
-              histograms.get(CoronaOps.KEY_CREATE.ordinal())
+              histograms.get(FreonOps.KEY_CREATE.ordinal())
                   .update(keyCreationDuration);
               keyCreationTime.getAndAdd(keyCreationDuration);
               long keyWriteStart = System.nanoTime();
@@ -750,7 +750,7 @@ public final class Corona extends Configured implements Tool {
               os.close();
               long keyWriteDuration = System.nanoTime() - keyWriteStart;
               threadKeyWriteTime += keyWriteDuration;
-              histograms.get(CoronaOps.KEY_WRITE.ordinal())
+              histograms.get(FreonOps.KEY_WRITE.ordinal())
                   .update(keyWriteDuration);
               totalBytesWritten.getAndAdd(keySize);
               numberOfKeysAdded.getAndIncrement();
@@ -780,7 +780,7 @@ public final class Corona extends Configured implements Tool {
 
   }
 
-  private final class CoronaJobInfo {
+  private final class FreonJobInfo {
 
     private String status;
     private String gitBaseRevision;
@@ -815,17 +815,17 @@ public final class Corona extends Configured implements Tool {
     private String deviationKeyWriteTime;
     private String[] tenQuantileKeyWriteTime;
 
-    private CoronaJobInfo() {
+    private FreonJobInfo() {
       this.status = exception ? "Failed" : "Success";
-      this.numOfVolumes = Corona.this.numOfVolumes;
-      this.numOfBuckets = Corona.this.numOfBuckets;
-      this.numOfKeys = Corona.this.numOfKeys;
-      this.numOfThreads = Corona.this.numOfThreads;
-      this.keySize = Corona.this.keySize;
-      this.mode = Corona.this.mode;
-      this.jobStartTime = Time.formatTime(Corona.this.jobStartTime);
-      this.replicationFactor = Corona.this.factor.name();
-      this.replicationType = Corona.this.type.name();
+      this.numOfVolumes = Freon.this.numOfVolumes;
+      this.numOfBuckets = Freon.this.numOfBuckets;
+      this.numOfKeys = Freon.this.numOfKeys;
+      this.numOfThreads = Freon.this.numOfThreads;
+      this.keySize = Freon.this.keySize;
+      this.mode = Freon.this.mode;
+      this.jobStartTime = Time.formatTime(Freon.this.jobStartTime);
+      this.replicationFactor = Freon.this.factor.name();
+      this.replicationType = Freon.this.type.name();
 
       long totalBytes =
           Long.parseLong(numOfVolumes) * Long.parseLong(numOfBuckets) * Long
@@ -833,7 +833,7 @@ public final class Corona extends Configured implements Tool {
       this.dataWritten = getInStorageUnits((double) totalBytes);
       this.totalThroughputPerSecond = getInStorageUnits(
           (totalBytes * 1.0) / TimeUnit.NANOSECONDS
-              .toSeconds(Corona.this.keyWriteTime.get() / threadPoolSize));
+              .toSeconds(Freon.this.keyWriteTime.get() / threadPoolSize));
     }
 
     private String getInStorageUnits(Double value) {
@@ -858,81 +858,81 @@ public final class Corona extends Configured implements Tool {
       return size + " " + unit;
     }
 
-    public CoronaJobInfo setGitBaseRevision(String gitBaseRevisionVal) {
+    public FreonJobInfo setGitBaseRevision(String gitBaseRevisionVal) {
       gitBaseRevision = gitBaseRevisionVal;
       return this;
     }
 
-    public CoronaJobInfo setExecTime(String execTimeVal) {
+    public FreonJobInfo setExecTime(String execTimeVal) {
       execTime = execTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setMeanKeyWriteTime(String deviationKeyWriteTimeVal) {
+    public FreonJobInfo setMeanKeyWriteTime(String deviationKeyWriteTimeVal) {
       this.meanKeyWriteTime = deviationKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setDeviationKeyWriteTime(
+    public FreonJobInfo setDeviationKeyWriteTime(
         String deviationKeyWriteTimeVal) {
       this.deviationKeyWriteTime = deviationKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setTenQuantileKeyWriteTime(
+    public FreonJobInfo setTenQuantileKeyWriteTime(
         String[] tenQuantileKeyWriteTimeVal) {
       this.tenQuantileKeyWriteTime = tenQuantileKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setMeanKeyCreateTime(String deviationKeyWriteTimeVal) {
+    public FreonJobInfo setMeanKeyCreateTime(String deviationKeyWriteTimeVal) {
       this.meanKeyCreateTime = deviationKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setDeviationKeyCreateTime(
+    public FreonJobInfo setDeviationKeyCreateTime(
         String deviationKeyCreateTimeVal) {
       this.deviationKeyCreateTime = deviationKeyCreateTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setTenQuantileKeyCreateTime(
+    public FreonJobInfo setTenQuantileKeyCreateTime(
         String[] tenQuantileKeyCreateTimeVal) {
       this.tenQuantileKeyCreateTime = tenQuantileKeyCreateTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setMeanBucketCreateTime(
+    public FreonJobInfo setMeanBucketCreateTime(
         String deviationKeyWriteTimeVal) {
       this.meanBucketCreateTime = deviationKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setDeviationBucketCreateTime(
+    public FreonJobInfo setDeviationBucketCreateTime(
         String deviationBucketCreateTimeVal) {
       this.deviationBucketCreateTime = deviationBucketCreateTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setTenQuantileBucketCreateTime(
+    public FreonJobInfo setTenQuantileBucketCreateTime(
         String[] tenQuantileBucketCreateTimeVal) {
       this.tenQuantileBucketCreateTime = tenQuantileBucketCreateTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setMeanVolumeCreateTime(
+    public FreonJobInfo setMeanVolumeCreateTime(
         String deviationKeyWriteTimeVal) {
       this.meanVolumeCreateTime = deviationKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setDeviationVolumeCreateTime(
+    public FreonJobInfo setDeviationVolumeCreateTime(
         String deviationVolumeCreateTimeVal) {
       this.deviationVolumeCreateTime = deviationVolumeCreateTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setTenQuantileVolumeCreateTime(
+    public FreonJobInfo setTenQuantileVolumeCreateTime(
         String[] tenQuantileVolumeCreateTimeVal) {
       this.tenQuantileVolumeCreateTime = tenQuantileVolumeCreateTimeVal;
       return this;

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/OzoneGettingStarted.md.vm

@@ -61,18 +61,18 @@ the background.
 
  - `docker-compose up -d`
 
-Now let us run some work load against ozone, to do that we will run corona.
+Now let us run some work load against ozone, to do that we will run freon.
 
 This will log into the datanode and run bash.
 
  - `docker-compose exec datanode bash`
  - `cd hadoop/bin`
 
-Now you can run the oz command shell or corona the ozone load generator.
+Now you can run the oz command shell or freon, the ozone load generator.
 
-This is the command to run corona.
+This is the command to run freon.
 
- - `./hdfs corona -mode offline -validateWrites -numOfVolumes 1 -numOfBuckets 10 -numOfKeys 100`
+ - `./hdfs freon -mode offline -validateWrites -numOfVolumes 1 -numOfBuckets 10 -numOfKeys 100`
 
 You can checkout the KSM UI to see the requests information.
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/OzoneOverview.md

@@ -22,11 +22,11 @@ keys. 
The following is a high-level overview of the core components of Ozone.
 The main elements of Ozone are
:
 
 ### Clients
-Ozone ships with a set of ready-made clients. They are 
Ozone CLI and Corona.

+Ozone ships with a set of ready-made clients. They are 
Ozone CLI and Freon.

 
     * [Ozone CLI](./OzoneCommandShell.html) is the command line interface like 'hdfs' command.

 
-    * Corona is a  load generation tool for Ozone.

+    * Freon is a  load generation tool for Ozone.

 
 ### REST Handler
 Ozone provides both an RPC (Remote Procedure Call) as well as a  REST

+ 27 - 27
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/tools/TestCorona.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/tools/TestFreon.java

@@ -36,9 +36,9 @@ import java.util.ArrayList;
 import java.util.List;
 
 /**
- * Tests Corona, with MiniOzoneCluster.
+ * Tests Freon, with MiniOzoneCluster.
  */
-public class TestCorona {
+public class TestFreon {
 
   private static MiniOzoneCluster cluster;
   private static OzoneConfiguration conf;
@@ -80,13 +80,13 @@ public class TestCorona {
     args.add("5");
     args.add("-numOfKeys");
     args.add("10");
-    Corona corona = new Corona(conf);
-    int res = ToolRunner.run(conf, corona,
+    Freon freon = new Freon(conf);
+    int res = ToolRunner.run(conf, freon,
         args.toArray(new String[0]));
-    Assert.assertEquals(2, corona.getNumberOfVolumesCreated());
-    Assert.assertEquals(10, corona.getNumberOfBucketsCreated());
-    Assert.assertEquals(100, corona.getNumberOfKeysAdded());
-    Assert.assertEquals(10240 - 36, corona.getKeyValueLength());
+    Assert.assertEquals(2, freon.getNumberOfVolumesCreated());
+    Assert.assertEquals(10, freon.getNumberOfBucketsCreated());
+    Assert.assertEquals(100, freon.getNumberOfKeysAdded());
+    Assert.assertEquals(10240 - 36, freon.getKeyValueLength());
     Assert.assertEquals(0, res);
   }
 
@@ -103,17 +103,17 @@ public class TestCorona {
     args.add("5");
     args.add("-numOfKeys");
     args.add("10");
-    Corona corona = new Corona(conf);
-    int res = ToolRunner.run(conf, corona,
+    Freon freon = new Freon(conf);
+    int res = ToolRunner.run(conf, freon,
         args.toArray(new String[0]));
     Assert.assertEquals(0, res);
-    Assert.assertEquals(2, corona.getNumberOfVolumesCreated());
-    Assert.assertEquals(10, corona.getNumberOfBucketsCreated());
-    Assert.assertEquals(100, corona.getNumberOfKeysAdded());
-    Assert.assertTrue(corona.getValidateWrites());
-    Assert.assertNotEquals(0, corona.getTotalKeysValidated());
-    Assert.assertNotEquals(0, corona.getSuccessfulValidationCount());
-    Assert.assertEquals(0, corona.getUnsuccessfulValidationCount());
+    Assert.assertEquals(2, freon.getNumberOfVolumesCreated());
+    Assert.assertEquals(10, freon.getNumberOfBucketsCreated());
+    Assert.assertEquals(100, freon.getNumberOfKeysAdded());
+    Assert.assertTrue(freon.getValidateWrites());
+    Assert.assertNotEquals(0, freon.getTotalKeysValidated());
+    Assert.assertNotEquals(0, freon.getSuccessfulValidationCount());
+    Assert.assertEquals(0, freon.getUnsuccessfulValidationCount());
     System.setOut(originalStream);
   }
 
@@ -130,12 +130,12 @@ public class TestCorona {
     args.add("10");
     args.add("-keySize");
     args.add("10240");
-    Corona corona = new Corona(conf);
-    int res = ToolRunner.run(conf, corona,
+    Freon freon = new Freon(conf);
+    int res = ToolRunner.run(conf, freon,
         args.toArray(new String[0]));
-    Assert.assertEquals(10, corona.getNumberOfVolumesCreated());
-    Assert.assertEquals(10, corona.getNumberOfBucketsCreated());
-    Assert.assertEquals(100, corona.getNumberOfKeysAdded());
+    Assert.assertEquals(10, freon.getNumberOfVolumesCreated());
+    Assert.assertEquals(10, freon.getNumberOfBucketsCreated());
+    Assert.assertEquals(100, freon.getNumberOfKeysAdded());
     Assert.assertEquals(0, res);
   }
 
@@ -154,12 +154,12 @@ public class TestCorona {
     args.add("10");
     args.add("-keySize");
     args.add("10240");
-    Corona corona = new Corona(conf);
-    int res = ToolRunner.run(conf, corona,
+    Freon freon = new Freon(conf);
+    int res = ToolRunner.run(conf, freon,
         args.toArray(new String[0]));
-    Assert.assertEquals(10, corona.getNumberOfVolumesCreated());
-    Assert.assertEquals(10, corona.getNumberOfBucketsCreated());
-    Assert.assertEquals(100, corona.getNumberOfKeysAdded());
+    Assert.assertEquals(10, freon.getNumberOfVolumesCreated());
+    Assert.assertEquals(10, freon.getNumberOfBucketsCreated());
+    Assert.assertEquals(100, freon.getNumberOfKeysAdded());
     Assert.assertEquals(0, res);
   }
 }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/OzoneTestDriver.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.test;
 
-import org.apache.hadoop.ozone.tools.Corona;
+import org.apache.hadoop.ozone.tools.Freon;
 import org.apache.hadoop.util.ProgramDriver;
 
 /**
@@ -35,7 +35,7 @@ public class OzoneTestDriver {
   public OzoneTestDriver(ProgramDriver pgd) {
     this.pgd = pgd;
     try {
-      pgd.addClass("corona", Corona.class,
+      pgd.addClass("freon", Freon.class,
           "Populates ozone with data.");
     } catch(Throwable e) {
       e.printStackTrace();