You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@bahir.apache.org by sourav-mazumder <gi...@git.apache.org> on 2016/11/15 23:49:25 UTC

[GitHub] bahir pull request #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS d...

GitHub user sourav-mazumder opened a pull request:

    https://github.com/apache/bahir/pull/27

    [BAHIR-75][WIP] Initital code delivery for WebHDFS data source

    Initial code delivery for webhdfs data source for Bahir. WE are still working on this. Comments are welcome.

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

    $ git pull https://github.com/sourav-mazumder/bahir spark-webhdfs

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

    https://github.com/apache/bahir/pull/27.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 #27
    
----
commit eee70dc6dac8aa1b3b9387aaa136250d340f2b46
Author: Sourav Mazumder <so...@gmail.com>
Date:   2016-11-15T23:34:14Z

    [BAHIR-75] Initital code delivery for WebHDFS 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.
---

[GitHub] bahir pull request #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS d...

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

    https://github.com/apache/bahir/pull/27#discussion_r88352457
  
    --- Diff: datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.bahir.datasource.webhdfs
    +
    +import java.text.SimpleDateFormat
    +import java.sql.{Timestamp, Date}
    +
    +import scala.collection.mutable.HashMap
    +
    +import org.apache.spark.sql.types.{DateType, TimestampType}
    +
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext}
    +
    +import org.apache.spark.sql.sources._
    +import org.apache.spark.sql.types.StructType
    +import scala.annotation.switch
    +
    +import org.apache.bahir.datasource.webhdfs.util._
    +import org.apache.bahir.datasource.webhdfs.csv._
    +
    +/**
    +	* This class contains functions for reading/writing data from/to remote webhdfs server in Spark DataSource
    +	* This function is written in line with the DataSource function in com.databricks.spark.csv. 
    +*/
    +
    +
    +class DefaultSource
    +  extends RelationProvider
    +  with SchemaRelationProvider
    +  with CreatableRelationProvider 
    +  with DataSourceRegister {
    +
    +  override def shortName() : String = "webhdfs"
    +
    +  private def checkPath(parameters: Map[String, String]): String = {
    +    	parameters.getOrElse("path", sys.error("'path' must be specified "))
    +  }
    +
    +  /**
    +   * Creates a new relation for data store in CSV given parameters.
    +   * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header'
    +   */
    +  override def createRelation(
    +      sqlContext: SQLContext,
    +      parameters: Map[String, String]): BaseRelation = {
    +    	createRelation(sqlContext, parameters, null)
    +  }
    +
    +  /**
    +   * Creates a new relation for data store in CSV given parameters and user supported schema.
    +   * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header'
    +   */
    --- End diff --
    
    yup, coded for `*.csv` only for now ...
    
    @sourav-mazumder do you plan on adding support for other file formats like `*.txt` and `*.json`?


---
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.
---

[GitHub] bahir pull request #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS d...

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

    https://github.com/apache/bahir/pull/27#discussion_r88345454
  
    --- Diff: datasource-webhdfs/pom.xml ---
    @@ -0,0 +1,83 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.bahir</groupId>
    +    <artifactId>bahir-parent_2.11</artifactId>
    +    <version>2.1.0-SNAPSHOT</version>
    +    <relativePath>../pom.xml</relativePath>
    +  </parent>
    +
    +  <groupId>org.apache.bahir</groupId>
    +  <artifactId>datasource-webhdfs_2.11</artifactId>
    +  <properties>
    +    <sbt.project.name>datasource-webhdfs</sbt.project.name>
    +  </properties>
    +  <packaging>jar</packaging>
    +  <name>Apache Bahir - Spark DataSource WebHDFS</name>
    +  <url>http://bahir.apache.org/</url>
    +
    +  <dependencies>
    +    <dependency>
    +      <groupId>org.scalaj</groupId>
    +      <artifactId>scalaj-http_2.11</artifactId>
    --- End diff --
    
    Please use _${scala.binary.version} instead of _2.11


---
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.
---

[GitHub] bahir issue #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS data sou...

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

    https://github.com/apache/bahir/pull/27
  
    This is now addressed in PR # 28. The new approach uses implementation of Hadoop File System which can be accessed from Spark using custom file system uri by setting SparkContext.hadoopConfiguration. This new approach can be used in a generic way for any type of file format (csv, json, et.) without any file format specific code.


---
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.
---

[GitHub] bahir pull request #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS d...

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

    https://github.com/apache/bahir/pull/27


---
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.
---

[GitHub] bahir issue #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS data sou...

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

    https://github.com/apache/bahir/pull/27
  
    @sourav-mazumder I have pushed a few more fixes to address Luciano's comments and opened new sub-tasks to track remaining work. These two should be our next priority:
    
    * [BAHIR-76](https://issues.apache.org/jira/browse/BAHIR-76): unit tests
    * [BAHIR-78](https://issues.apache.org/jira/browse/BAHIR-78): code clean-up
    
    
    I am able to test your new WebHDFS connector in a `spark-shell` now:
    
    ```Shell
    mvn clean install -pl datasource-webhdfs
    
    $SPARK_HOME/bin/spark-shell --packages org.apache.bahir:datasource-webhdfs_2.11:2.1.0-SNAPSHOT
    ```


---
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.
---

[GitHub] bahir pull request #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS d...

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

    https://github.com/apache/bahir/pull/27#discussion_r88360001
  
    --- Diff: pom.xml ---
    @@ -446,6 +447,7 @@
                   <exclude>.classpath</exclude>
                   <exclude>.project</exclude>
                   <exclude>**/dependency-reduced-pom.xml</exclude>
    +              <exclude>**/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister</exclude>
    --- End diff --
    
    @lresende even though [java.util.ServiceLoader](http://docs.oracle.com/javase/7/docs/api/java/util/ServiceLoader.html) ignores `#`-comment lines I did not see any examples of it being used in DataSourceRegister files of other projects &mdash; i.e. Spark does not include license comments in its 3 DataSourceRegister service provider configuration files. What's wrong with excluding it from the RAT license checks?


---
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.
---

[GitHub] bahir issue #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS data sou...

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

    https://github.com/apache/bahir/pull/27
  
    @sourav-mazumder  I pushed a fixes to your fork:
    
    * `README.md` correcting typos, add syntax highlighting
    *  `pom.xml` fix `RAT` excludes, include `META-INF/services` as resources in generated `jar` file so Spark can pick up your `webhdfs` data source
    
    I am in the process of fixing the 798 Scalastyle violations. The failed Scalastyle checks block the `mvn install` build which is supposed to install the new `datasource-webhdfs` module into the local Maven repository &mdash; a prerequisite to running `spark-submit` or `spark-shell` with the  `--packages datasource-webhdfs` option:
    ```Shell
    $ mvn install -DskipTests -pl datasource-webhdfs
    ```
    Output:
    ```                                                                    
    [INFO] ------------------------------------------------------------------------
    [INFO] Building Apache Bahir - Spark DataSource WebHDFS 2.1.0-SNAPSHOT
    [INFO] ------------------------------------------------------------------------
    ...
    [INFO] --- scalastyle-maven-plugin:0.8.0:check (default) @ datasource-webhdfs_2.11 ---
    ...
    Processed 10 file(s)
    Found 798 errors
    Found 0 warnings
    Found 0 infos
    Finished in 1912 ms
    [INFO] ------------------------------------------------------------------------
    [INFO] BUILD FAILURE
    [INFO] ------------------------------------------------------------------------
    [ERROR] Failed to execute goal org.scalastyle:scalastyle-maven-plugin:0.8.0:check (default) on project datasource-webhdfs_2.11: Failed during scalastyle execution: You have 798 Scalastyle violation(s). -> [Help 1]
    ...
    ```


---
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.
---

[GitHub] bahir pull request #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS d...

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

    https://github.com/apache/bahir/pull/27#discussion_r88347239
  
    --- Diff: pom.xml ---
    @@ -446,6 +447,7 @@
                   <exclude>.classpath</exclude>
                   <exclude>.project</exclude>
                   <exclude>**/dependency-reduced-pom.xml</exclude>
    +              <exclude>**/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister</exclude>
    --- End diff --
    
    Did we try just adding license file using # character as comment ? It should work and avoid rat exclusion


---
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.
---

[GitHub] bahir pull request #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS d...

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

    https://github.com/apache/bahir/pull/27#discussion_r88345357
  
    --- Diff: datasource-webhdfs/README.md ---
    @@ -0,0 +1,53 @@
    +A custom data source to read and write data from and to remote HDFS clusters using the [WebHDFS](https://hadoop.apache.org/docs/r2.7.3/hadoop-project-dist/hadoop-hdfs/WebHDFS.html) protocol. 
    +
    +## Linking
    +
    +Using SBT:
    +
    +```scala
    +libraryDependencies += "org.apache.bahir" %% "spark-datasource-webhdfs" % "2.1.0-SNAPSHOT"
    +```
    +
    +Using Maven:
    +
    +```xml
    +<dependency>
    +    <groupId>org.apache.bahir</groupId>
    +    <artifactId>spark-datasource-webhdfs_2.11</artifactId>
    +    <version>2.1.0-SNAPSHOT</version>
    +</dependency>
    +```
    +
    +This library can also be added to Spark jobs launched through `spark-shell` or `spark-submit` by using the `--packages` command line option.
    +For example, to include it when starting the spark shell:
    +
    +    $ bin/spark-shell --packages org.apache.bahir:spark-datasource-webhdfs_2.11:2.1.0-SNAPSHOT
    +
    +Unlike using `--jars`, using `--packages` ensures that this library and its dependencies will be added to the classpath.
    +The `--packages` argument can also be used with `bin/spark-submit`.
    +
    +This library is compiled for Scala 2.11 only, and intends to support Spark 2.0 onwards.
    --- End diff --
    
    This line above is not required. During our release process we do build 2.10 and 2.11 artifacts and make sure it is compatible with the Spark release being built with.


---
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.
---

[GitHub] bahir issue #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS data sou...

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

    https://github.com/apache/bahir/pull/27
  
    @sourav-mazumder I fixed the Scalastyle violations and added a few `//TODO` comments. But even beyond those there is much more work to do to simplify code and make consistent use of Scala idioms. We should open a separate code clean-up Task to track that work.


---
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.
---

[GitHub] bahir pull request #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS d...

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

    https://github.com/apache/bahir/pull/27#discussion_r88346170
  
    --- Diff: datasource-webhdfs/pom.xml ---
    @@ -0,0 +1,83 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.bahir</groupId>
    +    <artifactId>bahir-parent_2.11</artifactId>
    +    <version>2.1.0-SNAPSHOT</version>
    +    <relativePath>../pom.xml</relativePath>
    +  </parent>
    +
    +  <groupId>org.apache.bahir</groupId>
    +  <artifactId>datasource-webhdfs_2.11</artifactId>
    +  <properties>
    +    <sbt.project.name>datasource-webhdfs</sbt.project.name>
    +  </properties>
    +  <packaging>jar</packaging>
    +  <name>Apache Bahir - Spark DataSource WebHDFS</name>
    +  <url>http://bahir.apache.org/</url>
    +
    +  <dependencies>
    +    <dependency>
    +      <groupId>org.scalaj</groupId>
    +      <artifactId>scalaj-http_2.11</artifactId>
    --- End diff --
    
    Please use _${scala.binary.version} instead of _2.11


---
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.
---

[GitHub] bahir issue #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS data sou...

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

    https://github.com/apache/bahir/pull/27
  
    Current support for .csv will cover.txt as long as records are separated by
    '\n'. The field separator can be configured. By default it is assumed to be
    ','.
    
    In future support for .json and .xml would be added.
    
    Regards,
    Sourav
    
    On Wed, Nov 16, 2016 at 3:07 PM, Christian Kadner <no...@github.com>
    wrote:
    
    > *@ckadner* commented on this pull request.
    > ------------------------------
    >
    > In datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/
    > DefaultSource.scala <https://github.com/apache/bahir/pull/27>:
    >
    > > +  }
    > +
    > +  /**
    > +   * Creates a new relation for data store in CSV given parameters.
    > +   * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header'
    > +   */
    > +  override def createRelation(
    > +      sqlContext: SQLContext,
    > +      parameters: Map[String, String]): BaseRelation = {
    > +    	createRelation(sqlContext, parameters, null)
    > +  }
    > +
    > +  /**
    > +   * Creates a new relation for data store in CSV given parameters and user supported schema.
    > +   * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header'
    > +   */
    >
    > yup, coded for *.csv only for now ...
    >
    > @sourav-mazumder <https://github.com/sourav-mazumder> do you plan on
    > adding support for other file formats like *.txt and *.json?
    >
    > \u2014
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/bahir/pull/27>, or mute the thread
    > <https://github.com/notifications/unsubscribe-auth/AOQyx7ueDjp5725V0Nt0hhkX1z3x-6tUks5q-4ylgaJpZM4KzK9h>
    > .
    >



---
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.
---

[GitHub] bahir pull request #27: [BAHIR-75][WIP] Initital code delivery for WebHDFS d...

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

    https://github.com/apache/bahir/pull/27#discussion_r88346116
  
    --- Diff: datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.bahir.datasource.webhdfs
    +
    +import java.text.SimpleDateFormat
    +import java.sql.{Timestamp, Date}
    +
    +import scala.collection.mutable.HashMap
    +
    +import org.apache.spark.sql.types.{DateType, TimestampType}
    +
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext}
    +
    +import org.apache.spark.sql.sources._
    +import org.apache.spark.sql.types.StructType
    +import scala.annotation.switch
    +
    +import org.apache.bahir.datasource.webhdfs.util._
    +import org.apache.bahir.datasource.webhdfs.csv._
    +
    +/**
    +	* This class contains functions for reading/writing data from/to remote webhdfs server in Spark DataSource
    +	* This function is written in line with the DataSource function in com.databricks.spark.csv. 
    +*/
    +
    +
    +class DefaultSource
    +  extends RelationProvider
    +  with SchemaRelationProvider
    +  with CreatableRelationProvider 
    +  with DataSourceRegister {
    +
    +  override def shortName() : String = "webhdfs"
    +
    +  private def checkPath(parameters: Map[String, String]): String = {
    +    	parameters.getOrElse("path", sys.error("'path' must be specified "))
    +  }
    +
    +  /**
    +   * Creates a new relation for data store in CSV given parameters.
    +   * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header'
    +   */
    +  override def createRelation(
    +      sqlContext: SQLContext,
    +      parameters: Map[String, String]): BaseRelation = {
    +    	createRelation(sqlContext, parameters, null)
    +  }
    +
    +  /**
    +   * Creates a new relation for data store in CSV given parameters and user supported schema.
    +   * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header'
    +   */
    --- End diff --
    
    What is the relation with webHDFS and Csv ? Does it mean that we can only read csv files from the remote webHDFS at this time ?


---
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.
---