You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by gatorsmile <gi...@git.apache.org> on 2017/01/30 02:39:04 UTC

[GitHub] spark pull request #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

GitHub user gatorsmile opened a pull request:

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

    [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT case insensitive

    ### What changes were proposed in this pull request?
    Prior to Spark 2.1, the option names are case sensitive for all the formats. Since Spark 2.1, the option key names become case insensitive except the format `Text` and `LibSVM `. This PR is to fix these issues.
    
    Also, add a check to know whether the input option vector type is legal for `LibSVM`.
    
    ### How was this patch tested?
    Added test cases

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

    $ git pull https://github.com/gatorsmile/spark libSVMTextOptions

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

    https://github.com/apache/spark/pull/16737.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 #16737
    
----
commit 297f87aa3ac222c5c9f6e5fa6fcc3043b24bc0f9
Author: gatorsmile <ga...@gmail.com>
Date:   2017-01-30T02:26:36Z

    fix.

----


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    **[Test build #72238 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72238/testReport)** for PR 16737 at commit [`c2c145d`](https://github.com/apache/spark/commit/c2c145d55c586af703228d46ee04f5e0027739e8).


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    **[Test build #72143 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72143/testReport)** for PR 16737 at commit [`297f87a`](https://github.com/apache/spark/commit/297f87aa3ac222c5c9f6e5fa6fcc3043b24bc0f9).


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    **[Test build #72476 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72476/testReport)** for PR 16737 at commit [`07beaad`](https://github.com/apache/spark/commit/07beaad7808909fdfbb4b9592f6296801acf1956).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72528/
    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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

    https://github.com/apache/spark/pull/16737#discussion_r98602527
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMOptions.scala ---
    @@ -0,0 +1,51 @@
    +/*
    + * 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.spark.ml.source.libsvm
    +
    +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
    +
    +/**
    + * Options for the Text data source.
    --- End diff --
    
    `Options for the LibSVM data source.`?


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    **[Test build #72238 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72238/testReport)** for PR 16737 at commit [`c2c145d`](https://github.com/apache/spark/commit/c2c145d55c586af703228d46ee04f5e0027739e8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    **[Test build #72143 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72143/testReport)** for PR 16737 at commit [`297f87a`](https://github.com/apache/spark/commit/297f87aa3ac222c5c9f6e5fa6fcc3043b24bc0f9).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

    https://github.com/apache/spark/pull/16737#discussion_r99742193
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMOptions.scala ---
    @@ -0,0 +1,51 @@
    +/*
    + * 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.spark.ml.source.libsvm
    +
    +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
    +
    +/**
    + * Options for the LibSVM data source.
    + */
    +private[libsvm] class LibSVMOptions(@transient private val parameters: CaseInsensitiveMap)
    +  extends Serializable {
    +
    +  import LibSVMOptions._
    +
    +  def this(parameters: Map[String, String]) = this(new CaseInsensitiveMap(parameters))
    +
    +  /**
    +   * Number of features. If unspecified or nonpositive, the number of features will be determined
    +   * automatically at the cost of one additional pass.
    +   */
    +  val numFeatures = parameters.get(NUM_FEATURES).map(_.toInt)
    --- End diff --
    
    sure


---
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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

    https://github.com/apache/spark/pull/16737#discussion_r98836103
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMOptions.scala ---
    @@ -0,0 +1,51 @@
    +/*
    + * 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.spark.ml.source.libsvm
    +
    +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
    +
    +/**
    + * Options for the Text data source.
    --- End diff --
    
    uh. 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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/16737
  
    LGTM, merging to master!


---
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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

    https://github.com/apache/spark/pull/16737#discussion_r99732168
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala ---
    @@ -125,6 +124,25 @@ class TextSuite extends QueryTest with SharedSQLContext {
         }
       }
     
    +  test("case insensitive option") {
    +    val extraOptions = Map[String, String](
    +      "mApReDuCe.output.fileoutputformat.compress" -> "true",
    --- End diff --
    
    or is it because hadoopConf is case insensitive?


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/16737
  
    cc @cloud-fan @yhuai @dongjoon-hyun 


---
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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

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


---
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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

    https://github.com/apache/spark/pull/16737#discussion_r99730718
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMOptions.scala ---
    @@ -0,0 +1,51 @@
    +/*
    + * 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.spark.ml.source.libsvm
    +
    +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
    +
    +/**
    + * Options for the LibSVM data source.
    + */
    +private[libsvm] class LibSVMOptions(@transient private val parameters: CaseInsensitiveMap)
    +  extends Serializable {
    +
    +  import LibSVMOptions._
    +
    +  def this(parameters: Map[String, String]) = this(new CaseInsensitiveMap(parameters))
    +
    +  /**
    +   * Number of features. If unspecified or nonpositive, the number of features will be determined
    +   * automatically at the cost of one additional pass.
    +   */
    +  val numFeatures = parameters.get(NUM_FEATURES).map(_.toInt)
    --- End diff --
    
    how about `parameters.get(NUM_FEATURES).map(_.toInt).filter(_ > 0)`


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/16737
  
    Please hold on this PR. Found a serious bug to fix in case insensitive option support.


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16737
  
    +1. LGTM except one typo! @gatorsmile 


---
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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

    https://github.com/apache/spark/pull/16737#discussion_r99742158
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala ---
    @@ -125,6 +124,25 @@ class TextSuite extends QueryTest with SharedSQLContext {
         }
       }
     
    +  test("case insensitive option") {
    +    val extraOptions = Map[String, String](
    +      "mApReDuCe.output.fileoutputformat.compress" -> "true",
    --- End diff --
    
    The case is becomes lower cases when we build the DataSource. 


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72238/
    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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/16737
  
    I think `LIBSVM` and `TEXT` are the last two built-in sources that do not support case sensitivity. 


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    **[Test build #72528 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72528/testReport)** for PR 16737 at commit [`ec6eb6e`](https://github.com/apache/spark/commit/ec6eb6e1ff25511ba08d0b1e028a935812d4af13).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72143/
    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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    **[Test build #72476 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72476/testReport)** for PR 16737 at commit [`07beaad`](https://github.com/apache/spark/commit/07beaad7808909fdfbb4b9592f6296801acf1956).


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    Merged build finished. 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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16737
  
    Oh, then, are these the last piece of the case-sensitive options?


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72476/
    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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

    https://github.com/apache/spark/pull/16737#discussion_r99522959
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMOptions.scala ---
    @@ -0,0 +1,51 @@
    +/*
    + * 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.spark.ml.source.libsvm
    +
    +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
    +
    +/**
    + * Options for the LibSVM data source.
    + */
    +private[libsvm] class LibSVMOptions(@transient private val parameters: CaseInsensitiveMap)
    +  extends Serializable {
    +
    +  import LibSVMOptions._
    +
    +  def this(parameters: Map[String, String]) = this(new CaseInsensitiveMap(parameters))
    +
    +  /**
    +   * Number of features. If unspecified or nonpositive, the number of features will be determined
    +   * automatically at the cost of one additional pass.
    +   */
    +  val numFeatures = parameters.get(NUMFEATURES).map(_.toInt)
    +
    +  val isSparse = parameters.getOrElse(VECTORTYPE, SPARSEVECTORTYPE) match {
    +    case SPARSEVECTORTYPE => true
    +    case DENSEVECTORTYPE => false
    +    case o => throw new IllegalArgumentException(s"Invalid value `$o` for parameter " +
    +      s"`$VECTORTYPE`. Expected types are `sparse` and `dense`.")
    +  }
    +}
    +
    +private[libsvm] object LibSVMOptions {
    +  val NUMFEATURES = "numFeatures"
    --- End diff --
    
    nit: `NUM_FEATURES`


---
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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

    https://github.com/apache/spark/pull/16737#discussion_r99732105
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala ---
    @@ -125,6 +124,25 @@ class TextSuite extends QueryTest with SharedSQLContext {
         }
       }
     
    +  test("case insensitive option") {
    +    val extraOptions = Map[String, String](
    +      "mApReDuCe.output.fileoutputformat.compress" -> "true",
    --- End diff --
    
    where do we lower case these extra options?


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/16737
  
    what is the bug?


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    Merged build finished. 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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    Merged build finished. 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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/16737
  
    : ) When @sureshthalamati tried to fix the docker test failure in Oracle, he found the new code change in Spark 2.1 makes JDBC option key values always lower case. However, Oracle's connection properties are case sensitive. Thus, the changes caused the regression. 
    
    However, we realized the data source resolution always set the options to lower cases, as discussed in https://github.com/apache/spark/pull/14773. Thus, I think it is safe to do it, but we need to document the workaround if users hit this issue for JDBC sources. @sureshthalamati is doing it.
    
    Thus, this PR is ready to review. 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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    **[Test build #72528 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72528/testReport)** for PR 16737 at commit [`ec6eb6e`](https://github.com/apache/spark/commit/ec6eb6e1ff25511ba08d0b1e028a935812d4af13).


---
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 issue #16737: [SPARK-19397] [SQL] Make option names of LIBSVM and TEXT...

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

    https://github.com/apache/spark/pull/16737
  
    Merged build finished. 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