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

[GitHub] spark pull request: [WIP][SPARK-7025] Java-friendly InputSource AP...

GitHub user rxin opened a pull request:

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

    [WIP][SPARK-7025] Java-friendly InputSource API.

    Putting it out there so we can discuss.
    
    This pull request creates a new InputSource interface that is similar's Hadoop's InputFormat, as motivated in the JIRA ticket: https://issues.apache.org/jira/browse/SPARK-7025
    
    I also created an RDD implementation that takes in an InputSource and produces an RDD.

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

    $ git pull https://github.com/rxin/spark inputsource

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

    https://github.com/apache/spark/pull/5603.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 #5603
    
----
commit 4e99e2515066eb09dd1077bf55440ca713215b73
Author: Reynold Xin <rx...@databricks.com>
Date:   2015-04-21T05:18:19Z

    [SPARK-7025] Java-friendly InputSource API.

----


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94647608
  
      [Test build #30640 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30640/consoleFull) for   PR 5603 at commit [`f9d5c0b`](https://github.com/apache/spark/commit/f9d5c0b98b6e0f2498367aa0aac656bee1ff6647).


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94651444
  
      [Test build #30642 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30642/consoleFull) for   PR 5603 at commit [`2a2f213`](https://github.com/apache/spark/commit/2a2f2139b254fd754a3881dee872e652dfdde3cd).


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94874058
  
    That should still work with HadoopRDD. I can also create an HadoopInputFormatAdapter for InputSource.


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-107999296
  
     Merged build triggered.


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-107999369
  
    Merged build started.


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

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


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#discussion_r28838980
  
    --- Diff: core/src/main/java/org/apache/spark/io/RecordReader.java ---
    @@ -0,0 +1,53 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.io;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +
    +import org.apache.spark.annotation.Experimental;
    +
    +/**
    + * A reader that reads records in one by one.
    + */
    +@Experimental
    +public abstract class RecordReader<T> implements Closeable {
    +
    +  /**
    +   * Fetches the next record.
    +   *
    +   * @return true if we read a record.
    +   */
    +  public abstract boolean fetchNext() throws InterruptedException, IOException;
    +
    +  /**
    +   * Returns the current record. [[fetchNext]] must have been called before this.
    --- End diff --
    
    This should say that callers shouldn't rely on this value being unchanged after a subsequent call to `fetchNext`. Also, you should write `{@link #fetchNext}` rather than `[[fetchNext]]` for javadoc compatibility.


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94683865
  
      [Test build #30642 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30642/consoleFull) for   PR 5603 at commit [`2a2f213`](https://github.com/apache/spark/commit/2a2f2139b254fd754a3881dee872e652dfdde3cd).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public abstract class InputPartition `
      * `public abstract class InputSource<T> `
      * `public abstract class RecordReader<T> implements Closeable `
    
     * This patch does not change any dependencies.


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-108029749
  
    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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94646755
  
      [Test build #30639 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30639/consoleFull) for   PR 5603 at commit [`22c6306`](https://github.com/apache/spark/commit/22c63067993c0cdc43e88aafbc9febfa74acda96).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public abstract class InputPartition `
      * `public abstract class InputSource<T> `
      * `public abstract class RecordReader<T> implements Closeable `
    
     * This patch does not change any dependencies.


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94742427
  
    Would you still be able to reuse an existing `InputFormat` with this new model? that's going to still be useful in many cases. The only downside here I suppose is an extra layer of indirection and new API, to probably go alongside existing old methods that have to stick around. The upside is generality and simplicity. 


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-148824426
  
    @rxin, this PR probably isn't relevant anymore, given recent developments. Mind closing it for now?


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94645765
  
      [Test build #30638 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30638/consoleFull) for   PR 5603 at commit [`4e99e25`](https://github.com/apache/spark/commit/4e99e2515066eb09dd1077bf55440ca713215b73).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public abstract class InputPartition `
      * `public abstract class InputSource<T> `
      * `public abstract class RecordReader<T> implements Closeable `
    
     * This patch does not change any dependencies.


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-108029729
  
      [Test build #33997 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33997/consoleFull) for   PR 5603 at commit [`2a2f213`](https://github.com/apache/spark/commit/2a2f2139b254fd754a3881dee872e652dfdde3cd).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public abstract class InputPartition `
      * `public abstract class InputSource<T> `
      * `public abstract class RecordReader<T> implements Closeable `



---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94645768
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30638/
    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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94646187
  
      [Test build #30639 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30639/consoleFull) for   PR 5603 at commit [`22c6306`](https://github.com/apache/spark/commit/22c63067993c0cdc43e88aafbc9febfa74acda96).


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94647799
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30640/
    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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#discussion_r28986083
  
    --- Diff: core/src/main/scala/org/apache/spark/rdd/InputSourceRDD.scala ---
    @@ -0,0 +1,84 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.rdd
    +
    +import scala.reflect.ClassTag
    +import scala.util.control.NonFatal
    +
    +import org.apache.spark.{Logging, TaskContext, Partition, SparkContext}
    +import org.apache.spark.io.{InputPartition, InputSource}
    +
    +
    +private final class InputSourcePartition(val underlying: InputPartition, override val index: Int)
    +  extends Partition
    +
    +
    +private[spark] class InputSourceRDD[T: ClassTag](sc: SparkContext, inputSource: InputSource[T])
    +  extends RDD[T](sc, Nil) with Logging {
    +
    +  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
    +    val p = split.asInstanceOf[InputSourcePartition].underlying.asInstanceOf[InputPartition]
    +    val reader = inputSource.createRecordReader(p, context)
    +
    +    new Iterator[T] {
    +
    +      // Used to check whether we are done and make sure we only call underlying.close() once.
    +      private[this] var finished = false
    +
    +      context.addTaskCompletionListener(_ => close())
    +
    +      override def hasNext: Boolean = {
    +        if (!finished) {
    +          finished = !reader.fetchNext()
    --- End diff --
    
    Calling hasNext multiple times will advance the iterator by doing something similar to what Spark's own NextIterator does or extending NextIterator.


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94646758
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30639/
    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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-107999889
  
      [Test build #33997 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33997/consoleFull) for   PR 5603 at commit [`2a2f213`](https://github.com/apache/spark/commit/2a2f2139b254fd754a3881dee872e652dfdde3cd).


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94683970
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30642/
    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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-96767333
  
      [Test build #30997 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30997/consoleFull) for   PR 5603 at commit [`2a2f213`](https://github.com/apache/spark/commit/2a2f2139b254fd754a3881dee872e652dfdde3cd).


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94645100
  
      [Test build #30638 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30638/consoleFull) for   PR 5603 at commit [`4e99e25`](https://github.com/apache/spark/commit/4e99e2515066eb09dd1077bf55440ca713215b73).


---
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: [WIP][SPARK-7025] Java-friendly InputSource AP...

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

    https://github.com/apache/spark/pull/5603#issuecomment-94647793
  
      [Test build #30640 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30640/consoleFull) for   PR 5603 at commit [`f9d5c0b`](https://github.com/apache/spark/commit/f9d5c0b98b6e0f2498367aa0aac656bee1ff6647).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public abstract class InputPartition `
      * `public abstract class InputSource<T> `
      * `public abstract class RecordReader<T> implements Closeable `
    
     * This patch does not change any dependencies.


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