You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by jackylk <gi...@git.apache.org> on 2015/02/21 15:01:06 UTC

[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

GitHub user jackylk opened a pull request:

    https://github.com/apache/spark/pull/4714

    [SPARK-5939][MLLib] make FPGrowth example app take parameters

    Add parameter parsing in FPGrowth example app in Scala and Java
    And a sample data file is added in data/mllib folder

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/jackylk/spark parameter

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/4714.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4714
    
----
commit f0e4d10d11558d55995f02787b6904a9ab4b38a0
Author: Jacky Li <ja...@huawei.com>
Date:   2015-02-21T13:43:27Z

    make FPGrowth exampl app take parameters

commit 3bb74f6602c7697ced88de53bd008dad9b97fedb
Author: Jacky Li <ja...@huawei.com>
Date:   2015-02-21T13:43:54Z

    make FPGrowth exampl app take parameters

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4714#discussion_r25124130
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala ---
    @@ -18,29 +18,58 @@
     package org.apache.spark.examples.mllib
     
     import org.apache.spark.mllib.fpm.FPGrowth
    -import org.apache.spark.{SparkContext, SparkConf}
    +import org.apache.spark.{SparkConf, SparkContext}
    +import scopt.OptionParser
    --- End diff --
    
    organize imports (https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide#SparkCodeStyleGuide-Imports)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/4714#issuecomment-75376130
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27814/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4714#discussion_r25124128
  
    --- Diff: examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java ---
    @@ -25,32 +25,49 @@
     import org.apache.spark.SparkConf;
     import org.apache.spark.api.java.JavaRDD;
     import org.apache.spark.api.java.JavaSparkContext;
    +import org.apache.spark.api.java.function.Function;
     import org.apache.spark.mllib.fpm.FPGrowth;
     import org.apache.spark.mllib.fpm.FPGrowthModel;
     
     /**
      * Java example for mining frequent itemsets using FP-growth.
    + * Example usage:  ./bin/run-example org.apache.spark.examples.mllib.JavaFPGrowthExample
    + * ./data/mllib/sample_fpgrowth.txt
      */
     public class JavaFPGrowthExample {
     
       public static void main(String[] args) {
    +    String inputFile;
    +    double minSupport = 0.3;
    +    int numPartition = -1;
    +    if (args.length < 1) {
    +      System.err.println(
    +              "Usage: JavaKMeans <input_file> [minSupport] [numPartition]");
    +      System.exit(1);
    +    }
    +    inputFile = args[0];
    +    if (args.length >= 2) {
    +      minSupport = Double.parseDouble(args[1]);
    +    }
    +    if (args.length >= 3) {
    +      numPartition = Integer.parseInt(args[2]);
    +    }
    +
         SparkConf sparkConf = new SparkConf().setAppName("JavaFPGrowthExample");
         JavaSparkContext sc = new JavaSparkContext(sparkConf);
     
    +    JavaRDD<ArrayList<String>> transactions = sc.textFile(inputFile).map(
    +      new Function<String, ArrayList<String>>() {
    +      @Override
    --- End diff --
    
    fix indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4714#discussion_r25124132
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala ---
    @@ -18,29 +18,58 @@
     package org.apache.spark.examples.mllib
     
     import org.apache.spark.mllib.fpm.FPGrowth
    -import org.apache.spark.{SparkContext, SparkConf}
    +import org.apache.spark.{SparkConf, SparkContext}
    +import scopt.OptionParser
     
     /**
      * Example for mining frequent itemsets using FP-growth.
    + * Example usage: ./bin/run-example org.apache.spark.examples.mllib.FPGrowthExample
    + * --minSupport 0.8 --numPartition 2 ./data/mllib/sample_fpgrowth.txt
      */
     object FPGrowthExample {
     
    +  case class Params(
    +    input: String = null,
    +    minSupport: Double = 0.3,
    +    numPartition: Int = -1) extends AbstractParams[Params]
    +
       def main(args: Array[String]) {
    -    val conf = new SparkConf().setAppName("FPGrowthExample")
    +    val defaultParams = Params()
    +
    +    val parser = new OptionParser[Params]("FPGrowth") {
    --- End diff --
    
    `FPGrowth` -> `FPGrowthExample`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4714#discussion_r25124134
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala ---
    @@ -18,29 +18,58 @@
     package org.apache.spark.examples.mllib
     
     import org.apache.spark.mllib.fpm.FPGrowth
    -import org.apache.spark.{SparkContext, SparkConf}
    +import org.apache.spark.{SparkConf, SparkContext}
    +import scopt.OptionParser
     
     /**
      * Example for mining frequent itemsets using FP-growth.
    + * Example usage: ./bin/run-example org.apache.spark.examples.mllib.FPGrowthExample
    + * --minSupport 0.8 --numPartition 2 ./data/mllib/sample_fpgrowth.txt
      */
     object FPGrowthExample {
     
    +  case class Params(
    +    input: String = null,
    +    minSupport: Double = 0.3,
    +    numPartition: Int = -1) extends AbstractParams[Params]
    +
       def main(args: Array[String]) {
    -    val conf = new SparkConf().setAppName("FPGrowthExample")
    +    val defaultParams = Params()
    +
    +    val parser = new OptionParser[Params]("FPGrowth") {
    +      head("FPGrowth: an example FP-growth app.")
    +      opt[Double]("minSupport")
    +        .text(s"minimal support level, default: ${defaultParams.minSupport}")
    +        .action((x, c) => c.copy(minSupport = x))
    +      opt[Int]("numPartition")
    +        .text(s"number of partition, default: ${defaultParams.numPartition}")
    +        .action((x, c) => c.copy(numPartition = x))
    +      arg[String]("<input>")
    +        .text("input paths to input data set")
    +        .required()
    +        .action((x, c) => c.copy(input = x))
    +    }
    +
    +    parser.parse(args, defaultParams).map { params =>
    +      run(params)
    +    }.getOrElse {
    +      sys.exit(1)
    +    }
    +  }
    +
    +  def run(params: Params) {
    +    val conf = new SparkConf().setAppName(s"FPGrowthExample with $params")
         val sc = new SparkContext(conf)
    +    val transactions = sc.textFile(params.input).map(_.split(" ")).cache()
    +
    +    println(s"Number of transactions: ${transactions.count}")
    +
    +    val model = new FPGrowth()
    +      .setMinSupport(params.minSupport)
    +      .setNumPartitions(params.numPartition)
    +      .run(transactions)
     
    -    // TODO: Read a user-specified input file.
    -    val transactions = sc.parallelize(Seq(
    -      "r z h k p",
    -      "z y x w v u t s",
    -      "s x o n r",
    -      "x z y m t s q e",
    -      "z",
    -      "x z y r q t p").map(_.split(" ")), numSlices = 2)
    -
    -    val fpg = new FPGrowth()
    -      .setMinSupport(0.3)
    -    val model = fpg.run(transactions)
    +    println(s"Number of frequent itemsets: ${model.freqItemsets.count}")
    --- End diff --
    
    `count` -> `count()`. We use `()` if it is an action.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4714#discussion_r25124133
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala ---
    @@ -18,29 +18,58 @@
     package org.apache.spark.examples.mllib
     
     import org.apache.spark.mllib.fpm.FPGrowth
    -import org.apache.spark.{SparkContext, SparkConf}
    +import org.apache.spark.{SparkConf, SparkContext}
    +import scopt.OptionParser
     
     /**
      * Example for mining frequent itemsets using FP-growth.
    + * Example usage: ./bin/run-example org.apache.spark.examples.mllib.FPGrowthExample
    + * --minSupport 0.8 --numPartition 2 ./data/mllib/sample_fpgrowth.txt
      */
     object FPGrowthExample {
     
    +  case class Params(
    +    input: String = null,
    +    minSupport: Double = 0.3,
    +    numPartition: Int = -1) extends AbstractParams[Params]
    +
       def main(args: Array[String]) {
    -    val conf = new SparkConf().setAppName("FPGrowthExample")
    +    val defaultParams = Params()
    +
    +    val parser = new OptionParser[Params]("FPGrowth") {
    +      head("FPGrowth: an example FP-growth app.")
    +      opt[Double]("minSupport")
    +        .text(s"minimal support level, default: ${defaultParams.minSupport}")
    +        .action((x, c) => c.copy(minSupport = x))
    +      opt[Int]("numPartition")
    +        .text(s"number of partition, default: ${defaultParams.numPartition}")
    +        .action((x, c) => c.copy(numPartition = x))
    +      arg[String]("<input>")
    +        .text("input paths to input data set")
    --- End diff --
    
    Add more description about the file format.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4714#discussion_r25124131
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala ---
    @@ -18,29 +18,58 @@
     package org.apache.spark.examples.mllib
     
     import org.apache.spark.mllib.fpm.FPGrowth
    -import org.apache.spark.{SparkContext, SparkConf}
    +import org.apache.spark.{SparkConf, SparkContext}
    +import scopt.OptionParser
     
     /**
      * Example for mining frequent itemsets using FP-growth.
    + * Example usage: ./bin/run-example org.apache.spark.examples.mllib.FPGrowthExample
    --- End diff --
    
    Same here, remove `org.apache.spark.examples.`. If the options doesn't fit into one line, use `\` at the end of the first line and indent the second.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4714#issuecomment-75372902
  
      [Test build #27814 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27814/consoleFull) for   PR 4714 at commit [`3bb74f6`](https://github.com/apache/spark/commit/3bb74f6602c7697ced88de53bd008dad9b97fedb).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4714#issuecomment-75386009
  
      [Test build #27821 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27821/consoleFull) for   PR 4714 at commit [`8c478b3`](https://github.com/apache/spark/commit/8c478b317b2bf70768db2ad09da1e34c22c6f50b).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4714#issuecomment-75389943
  
      [Test build #27821 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27821/consoleFull) for   PR 4714 at commit [`8c478b3`](https://github.com/apache/spark/commit/8c478b317b2bf70768db2ad09da1e34c22c6f50b).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  case class Params(`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on the pull request:

    https://github.com/apache/spark/pull/4714#issuecomment-75580799
  
    LGTM. Merged into master and branch-1.3. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/4714


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4714#issuecomment-75376126
  
      [Test build #27814 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27814/consoleFull) for   PR 4714 at commit [`3bb74f6`](https://github.com/apache/spark/commit/3bb74f6602c7697ced88de53bd008dad9b97fedb).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  case class Params(`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4714#discussion_r25124126
  
    --- Diff: examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java ---
    @@ -25,32 +25,49 @@
     import org.apache.spark.SparkConf;
     import org.apache.spark.api.java.JavaRDD;
     import org.apache.spark.api.java.JavaSparkContext;
    +import org.apache.spark.api.java.function.Function;
     import org.apache.spark.mllib.fpm.FPGrowth;
     import org.apache.spark.mllib.fpm.FPGrowthModel;
     
     /**
      * Java example for mining frequent itemsets using FP-growth.
    + * Example usage:  ./bin/run-example org.apache.spark.examples.mllib.JavaFPGrowthExample
    --- End diff --
    
    `run-example` doesn't need `org.apache.spark.examples.` So this could be a one-liner:
    
    ~~~
    .bin/run-example mllib.JavaFPGrowthExample ./data/mllib/sample_fpgrowth.txt
    ~~~


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/4714#issuecomment-75389944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27821/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-5939][MLLib] make FPGrowth example app ...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4714#discussion_r25124127
  
    --- Diff: examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java ---
    @@ -25,32 +25,49 @@
     import org.apache.spark.SparkConf;
     import org.apache.spark.api.java.JavaRDD;
     import org.apache.spark.api.java.JavaSparkContext;
    +import org.apache.spark.api.java.function.Function;
     import org.apache.spark.mllib.fpm.FPGrowth;
     import org.apache.spark.mllib.fpm.FPGrowthModel;
     
     /**
      * Java example for mining frequent itemsets using FP-growth.
    + * Example usage:  ./bin/run-example org.apache.spark.examples.mllib.JavaFPGrowthExample
    + * ./data/mllib/sample_fpgrowth.txt
      */
     public class JavaFPGrowthExample {
     
       public static void main(String[] args) {
    +    String inputFile;
    +    double minSupport = 0.3;
    +    int numPartition = -1;
    +    if (args.length < 1) {
    +      System.err.println(
    +              "Usage: JavaKMeans <input_file> [minSupport] [numPartition]");
    --- End diff --
    
    Fix indentation and change `JavaKMeans` to `mllib.JavaFPGrowthExample`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org