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

[GitHub] spark pull request #17081: [SPAKR-18726][SQL][WIP]resolveRelation for FileFo...

GitHub user windpiger opened a pull request:

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

    [SPAKR-18726][SQL][WIP]resolveRelation for FileFormat DataSource don't need to  listFiles twice

    ## What changes were proposed in this pull request?
    
    Currently when we resolveRelation for a `FileFormat DataSource` without providing user schema, it will execute `listFiles`  twice in `InMemoryFileIndex` during `resolveRelation`.
    
    This PR add a `FileStatusCache` for DataSource, this can avoid listFiles twice.
    
    But there is a bug in `InMemoryFileIndex` see [SPARK-19748](https://github.com/apache/spark/pull/17079), so this pr should be after SPARK-19748.
    
    
    ## How was this patch tested?
    N/A

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

    $ git pull https://github.com/windpiger/spark resolveDataSourceScanFilesTwice

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

    https://github.com/apache/spark/pull/17081.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 #17081
    
----
commit 0082b7633e8f84fe5cafa0362cd45cce4cfee459
Author: windpiger <so...@outlook.com>
Date:   2017-02-27T08:04:30Z

    [SPAKR-18726][SQL]resolveRelation for FileFormate DataSource don't need to  listFiles twice

----


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73724 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73724/testReport)** for PR 17081 at commit [`a8c1dea`](https://github.com/apache/spark/commit/a8c1deab0fc8e59863bf4a3d3b551f77fbebbc6d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class ResolvedDataSourceSuite extends SparkFunSuite with SharedSQLContext `


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    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 pull request #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

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/17081#discussion_r103862351
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -86,7 +86,7 @@ case class DataSource(
       lazy val providingClass: Class[_] = DataSource.lookupDataSource(className)
       lazy val sourceInfo: SourceInfo = sourceSchema()
       private val caseInsensitiveOptions = CaseInsensitiveMap(options)
    -
    +  private lazy val fileStatusCache = FileStatusCache.getOrCreate(sparkSession)
    --- End diff --
    
    what's the life time of this cache?


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73758/
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73758 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73758/testReport)** for PR 17081 at commit [`28c8158`](https://github.com/apache/spark/commit/28c8158a7c9d7acdbf2a07ef66ace46c1215979f).
     * 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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73733/
    Test FAILed.


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73791/
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73758 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73758/testReport)** for PR 17081 at commit [`28c8158`](https://github.com/apache/spark/commit/28c8158a7c9d7acdbf2a07ef66ace46c1215979f).


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

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

    https://github.com/apache/spark/pull/17081#discussion_r103900016
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -122,7 +122,7 @@ case class DataSource(
             val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
             SparkHadoopUtil.get.globPathIfNecessary(qualified)
           }.toArray
    -      new InMemoryFileIndex(sparkSession, globbedPaths, options, None)
    +      new InMemoryFileIndex(sparkSession, globbedPaths, options, None, fileStatusCache)
    --- End diff --
    
    I have make the it local only in the no streaming FileFormat match case~


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73726/
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

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/17081#discussion_r103862631
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -364,7 +364,12 @@ case class DataSource(
                 catalogTable.get,
                 catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize))
             } else {
    -          new InMemoryFileIndex(sparkSession, globbedPaths, options, Some(partitionSchema))
    --- End diff --
    
    I'd like to create file status cache as a local variable, pass it to `getOrInferFileFormatSchema`, then use it here. It's much easier to reason about the lifetime of this cache by this way.


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73716 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73716/testReport)** for PR 17081 at commit [`f79f12c`](https://github.com/apache/spark/commit/f79f12c552ee1721295c347744fc5f92f048c74b).
     * This patch **fails Spark unit 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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat ...

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

    https://github.com/apache/spark/pull/17081#discussion_r104073970
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -106,6 +106,7 @@ case class DataSource(
        *     be any further inference in any triggers.
        *
        * @param format the file format object for this DataSource
    +   * @param fileStatusCache fileStatusCache for InMemoryFileIndex
    --- End diff --
    
    `@param fileStatusCache the shared cache for file statuses to speed up listing`


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

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

    https://github.com/apache/spark/pull/17081#discussion_r103861424
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -364,7 +364,12 @@ case class DataSource(
                 catalogTable.get,
                 catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize))
             } else {
    -          new InMemoryFileIndex(sparkSession, globbedPaths, options, Some(partitionSchema))
    +          new InMemoryFileIndex(
    +            sparkSession,
    +            globbedPaths,
    +            options,
    +            Some(partitionSchema),
    +            fileStatusCache)
    --- End diff --
    
    ```Scala
              new InMemoryFileIndex(
                sparkSession, globbedPaths, options, Some(partitionSchema), fileStatusCache)
    ```
    
    This is also valid


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Why you closed it?


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat ...

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

    https://github.com/apache/spark/pull/17081#discussion_r104065615
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -109,7 +109,9 @@ case class DataSource(
        * @return A pair of the data schema (excluding partition columns) and the schema of the partition
        *         columns.
        */
    -  private def getOrInferFileFormatSchema(format: FileFormat): (StructType, StructType) = {
    +  private def getOrInferFileFormatSchema(
    +      format: FileFormat,
    +      fileStatusCache: FileStatusCache = NoopCache): (StructType, StructType) = {
    --- End diff --
    
    ok, 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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

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


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    retest this please


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

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


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73798/
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    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 pull request #17081: [SPARK-18726][SQL]resolveRelation for FileFormat ...

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

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


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73759 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73759/testReport)** for PR 17081 at commit [`92618b3`](https://github.com/apache/spark/commit/92618b3ad67c899e681a9923ad9abc5a7f2c7897).
     * 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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

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


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    cc @cloud-fan @gatorsmile  could you help to review this?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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73715 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73715/testReport)** for PR 17081 at commit [`f1da0a4`](https://github.com/apache/spark/commit/f1da0a4cf457f4efb6128beca3c08ccf95ef37a0).
     * This patch **fails Spark unit 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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    retest this please


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    LGTM


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73500/
    Test FAILed.


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Merged build finished. Test FAILed.


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73716/
    Test FAILed.


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Merged build finished. Test FAILed.


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73726 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73726/testReport)** for PR 17081 at commit [`60fa037`](https://github.com/apache/spark/commit/60fa03757d223f833e2fa161326a48a9015d4c6c).


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Merged build finished. Test FAILed.


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat ...

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

    https://github.com/apache/spark/pull/17081#discussion_r104075579
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -106,6 +106,7 @@ case class DataSource(
        *     be any further inference in any triggers.
        *
        * @param format the file format object for this DataSource
    +   * @param fileStatusCache fileStatusCache for InMemoryFileIndex
    --- End diff --
    
    ok 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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

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


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73759/
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Merged build finished. Test FAILed.


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73791 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73791/testReport)** for PR 17081 at commit [`92618b3`](https://github.com/apache/spark/commit/92618b3ad67c899e681a9923ad9abc5a7f2c7897).
     * 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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73798 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73798/testReport)** for PR 17081 at commit [`3e495a7`](https://github.com/apache/spark/commit/3e495a73fb01fe118c93422e87d1be1f66e78678).


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73715/
    Test FAILed.


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73726 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73726/testReport)** for PR 17081 at commit [`60fa037`](https://github.com/apache/spark/commit/60fa03757d223f833e2fa161326a48a9015d4c6c).
     * 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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Please remove `[FOLLOW-UP]` from the PR title. 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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73791 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73791/testReport)** for PR 17081 at commit [`92618b3`](https://github.com/apache/spark/commit/92618b3ad67c899e681a9923ad9abc5a7f2c7897).


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

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

    https://github.com/apache/spark/pull/17081#discussion_r103864206
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -364,7 +364,12 @@ case class DataSource(
                 catalogTable.get,
                 catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize))
             } else {
    -          new InMemoryFileIndex(sparkSession, globbedPaths, options, Some(partitionSchema))
    --- End diff --
    
    ok, I think it is more reasonable~ 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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73793 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73793/testReport)** for PR 17081 at commit [`f6ec4fe`](https://github.com/apache/spark/commit/f6ec4fe020c153ca0e84afd3cbb5def22d9b6651).
     * 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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

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


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

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

    https://github.com/apache/spark/pull/17081#discussion_r104063471
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -109,7 +109,9 @@ case class DataSource(
        * @return A pair of the data schema (excluding partition columns) and the schema of the partition
        *         columns.
        */
    -  private def getOrInferFileFormatSchema(format: FileFormat): (StructType, StructType) = {
    +  private def getOrInferFileFormatSchema(
    +      format: FileFormat,
    +      fileStatusCache: FileStatusCache = NoopCache): (StructType, StructType) = {
    --- End diff --
    
    Please update the function description with a new `@parm`


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73759 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73759/testReport)** for PR 17081 at commit [`92618b3`](https://github.com/apache/spark/commit/92618b3ad67c899e681a9923ad9abc5a7f2c7897).


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

Posted by windpiger <gi...@git.apache.org>.
GitHub user windpiger reopened a pull request:

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

    [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFormat DataSource don't need to  listFiles twice

    ## What changes were proposed in this pull request?
    
    Currently when we resolveRelation for a `FileFormat DataSource` without providing user schema, it will execute `listFiles`  twice in `InMemoryFileIndex` during `resolveRelation`.
    
    This PR add a `FileStatusCache` for DataSource, this can avoid listFiles twice.
    
    But there is a bug in `InMemoryFileIndex` see:
     [SPARK-19748](https://github.com/apache/spark/pull/17079)
     [SPARK-19761](https://github.com/apache/spark/pull/17093), 
    so this pr should be after SPARK-19748/ SPARK-19761.
    
    
    ## How was this patch tested?
    unit test added

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

    $ git pull https://github.com/windpiger/spark resolveDataSourceScanFilesTwice

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

    https://github.com/apache/spark/pull/17081.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 #17081
    
----
commit 0082b7633e8f84fe5cafa0362cd45cce4cfee459
Author: windpiger <so...@outlook.com>
Date:   2017-02-27T08:04:30Z

    [SPAKR-18726][SQL]resolveRelation for FileFormate DataSource don't need to  listFiles twice

commit 6b5454ad0104459565febb520fa22ef30bdb8368
Author: windpiger <so...@outlook.com>
Date:   2017-02-27T08:39:45Z

    add test case

commit f1da0a4cf457f4efb6128beca3c08ccf95ef37a0
Author: windpiger <so...@outlook.com>
Date:   2017-02-27T23:59:34Z

    fix a style

commit f79f12c552ee1721295c347744fc5f92f048c74b
Author: windpiger <so...@outlook.com>
Date:   2017-03-01T22:49:13Z

    Merge branch 'master' into resolveDataSourceScanFilesTwice

commit a8c1deab0fc8e59863bf4a3d3b551f77fbebbc6d
Author: windpiger <so...@outlook.com>
Date:   2017-03-02T01:50:30Z

    fix test failed

commit 60fa03757d223f833e2fa161326a48a9015d4c6c
Author: windpiger <so...@outlook.com>
Date:   2017-03-02T04:49:08Z

    add a lazy

commit 9a73947efea334ba0cfc5b5508003807a93ff806
Author: windpiger <so...@outlook.com>
Date:   2017-03-02T06:49:44Z

    fix code style

commit 850094cd3b77f6ecf33caf88532920e73de976f4
Author: windpiger <so...@outlook.com>
Date:   2017-03-02T06:54:38Z

    Merge branch 'master' of github.com:apache/spark into resolveDataSourceScanFilesTwice

commit c39eb26da38f9d92e3871814be446c8d911be890
Author: windpiger <so...@outlook.com>
Date:   2017-03-02T11:03:18Z

    make filestatuscache local var

commit f3332cb870ae2be9383969de07a07c8761230e8b
Author: windpiger <so...@outlook.com>
Date:   2017-03-02T11:04:55Z

    modify a test case

commit 9cadd4168041fd859cc1e4b8396e5ed514129bff
Author: windpiger <so...@outlook.com>
Date:   2017-03-02T11:05:24Z

    modify a test case

commit 28c8158a7c9d7acdbf2a07ef66ace46c1215979f
Author: windpiger <so...@outlook.com>
Date:   2017-03-02T11:06:40Z

    modify a test case

commit 92618b3ad67c899e681a9923ad9abc5a7f2c7897
Author: windpiger <so...@outlook.com>
Date:   2017-03-02T11:07:10Z

    remove an empty line

----


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73793/
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73724/
    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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

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

    https://github.com/apache/spark/pull/17081#discussion_r103861992
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -122,7 +122,7 @@ case class DataSource(
             val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
             SparkHadoopUtil.get.globPathIfNecessary(qualified)
           }.toArray
    -      new InMemoryFileIndex(sparkSession, globbedPaths, options, None)
    +      new InMemoryFileIndex(sparkSession, globbedPaths, options, None, fileStatusCache)
    --- End diff --
    
    This also impacts the streaming code path. If it is fine to streaming, the code changes look good to me.


---
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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73798 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73798/testReport)** for PR 17081 at commit [`3e495a7`](https://github.com/apache/spark/commit/3e495a73fb01fe118c93422e87d1be1f66e78678).
     * 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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73500 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73500/testReport)** for PR 17081 at commit [`6b5454a`](https://github.com/apache/spark/commit/6b5454ad0104459565febb520fa22ef30bdb8368).


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    oh...sorry \uff0c I don't know when I close it... 


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73500 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73500/testReport)** for PR 17081 at commit [`6b5454a`](https://github.com/apache/spark/commit/6b5454ad0104459565febb520fa22ef30bdb8368).
     * This patch **fails Scala style 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 #17081: [SPARK-18726][SQL]resolveRelation for FileFormat DataSou...

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

    https://github.com/apache/spark/pull/17081
  
    thanks, 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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

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

    https://github.com/apache/spark/pull/17081#discussion_r103859650
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -364,7 +364,8 @@ case class DataSource(
                 catalogTable.get,
                 catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize))
             } else {
    -          new InMemoryFileIndex(sparkSession, globbedPaths, options, Some(partitionSchema))
    +          new InMemoryFileIndex(sparkSession, globbedPaths, options, Some(partitionSchema),
    +            fileStatusCache)
    --- End diff --
    
    Nit: indent issue


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for FileFor...

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

    https://github.com/apache/spark/pull/17081
  
    **[Test build #73733 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73733/testReport)** for PR 17081 at commit [`9a73947`](https://github.com/apache/spark/commit/9a73947efea334ba0cfc5b5508003807a93ff806).


---
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 #17081: [SPARK-18726][SQL][FOLLOW-UP]resolveRelation for ...

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

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


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