[GitHub] spark pull request: [SPARK-3366][MLLIB]Compute best splits distrib...

2014-10-01 Thread chouqin
Github user chouqin commented on the pull request:

https://github.com/apache/spark/pull/2595#issuecomment-57423431
  
I don't know why unit test for pyspark has failed, I have tested in Scala 
using the same test data, and it passes. For example here is code I wrote:

```scala
   // this data is correspond to data defined at pyspark/mllib/tests.py: 
line 125
val arr = new Array[LabeledPoint](4)
arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0))
arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0))
arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0))
arr(3) = new LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))
val input = sc.parallelize(arr)

// these parameters are correspond to DecisionTree.trainClassifier at 
pyspark/mllib/tests.py: line 154
val strategy = new Strategy(algo = Classification, impurity = Gini, 
maxDepth=5,
  numClassesForClassification = 2, categoricalFeaturesInfo = Map(0 - 
3))
val model = DecisionTree.train(input, strategy)

   // these asserts will pass
assert(model.predict(arr(0).features) == 0.0)
assert(model.predict(arr(1).features) == 1.0)
assert(model.predict(arr(2).features) == 0.0)
assert(model.predict(arr(3).features) == 1.0)
```


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

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



[GitHub] spark pull request: [SPARK-3747] TaskResultGetter could incorrectl...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2599#issuecomment-57423555
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/220/consoleFull)
 for   PR 2599 at commit 
[`0e8d44c`](https://github.com/apache/spark/commit/0e8d44ccd930a655b85b184889c965d2c2e68694).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-3748] Log thread name in unit test logs

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2600#issuecomment-57423550
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/221/consoleFull)
 for   PR 2600 at commit 
[`83ffe88`](https://github.com/apache/spark/commit/83ffe888ce7cf230b4b1721bbf1de7bdfffa7c2c).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-3747] TaskResultGetter could incorrectl...

2014-10-01 Thread kayousterhout
Github user kayousterhout commented on the pull request:

https://github.com/apache/spark/pull/2599#issuecomment-57423834
  
As discussed offline, this looks good if you add a comment about why 
NonFatal.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3747] TaskResultGetter could incorrectl...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2599#issuecomment-57424163
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21095/consoleFull)
 for   PR 2599 at commit 
[`a74c04d`](https://github.com/apache/spark/commit/a74c04d5770b9d120f71df91563251f680ae37d3).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-3366][MLLIB]Compute best splits distrib...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3366][MLLIB]Compute best splits distrib...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2595#issuecomment-57424470
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21090/consoleFull)
 for   PR 2595 at commit 
[`c32636e`](https://github.com/apache/spark/commit/c32636e19e015d42513910c99173d2f212b0f366).
 * This patch **fails** 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService

2014-10-01 Thread aarondav
Github user aarondav commented on a diff in the pull request:

https://github.com/apache/spark/pull/2330#discussion_r18263375
  
--- Diff: core/src/main/scala/org/apache/spark/SparkEnv.scala ---
@@ -234,7 +236,12 @@ object SparkEnv extends Logging {
 
 val shuffleMemoryManager = new ShuffleMemoryManager(conf)
 
-val blockTransferService = new NioBlockTransferService(conf, 
securityManager)
+// TODO(rxin): Config option based on class name, similar to shuffle 
mgr and compression codec.
+val blockTransferService = if 
(conf.getBoolean(spark.shuffle.use.netty, false)) {
--- End diff --

It'd be better if we could go ahead and make the config stable, it's always 
painful for people to update later. Even if it's just like

```scala
val blockTransferService = conf.getString(spark.shuffle.transferService, 
NIO) match {
  case NETTY = new NettyBlockTransferService(conf)
  case NIO = new NioBlockTransferService(conf, securityManager)
  case s = throw new UnsupportedOperationException(Unknown transfer 
service:  + s)
}
```

Potentially less pain for when we address the TODO at a later date.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService

2014-10-01 Thread aarondav
Github user aarondav commented on a diff in the pull request:

https://github.com/apache/spark/pull/2330#discussion_r18263398
  
--- Diff: 
core/src/main/scala/org/apache/spark/network/BlockDataManager.scala ---
@@ -20,14 +20,14 @@ package org.apache.spark.network
 import org.apache.spark.storage.StorageLevel
 
 
+private[spark]
 trait BlockDataManager {
--- End diff --

Did you add this class at some point? Either way, would you mind adding a 
class comment? It's not clear how it differs from all the other block-related 
managers.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3343] [SQL] Add serde support for CTAS

2014-10-01 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/2570#discussion_r18263442
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala 
---
@@ -32,6 +32,54 @@ case class Nested3(f3: Int)
  * valid, but Hive currently cannot execute it.
  */
 class SQLQuerySuite extends QueryTest {
+  test(CTAS with serde) {
+sql(CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, 
value).collect
+sql(
+  CREATE TABLE ctas2
+| ROW FORMAT SERDE 
org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
+| STORED AS RCFile AS
+|   SELECT key, value
+|   FROM src
+|   ORDER BY key, value.stripMargin).collect
--- End diff --

That's a good question, I will add the `explain` command to verify if the 
properties are correctly set.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: SPARK-2621. Update task InputMetrics increment...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2087#issuecomment-57425652
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21091/consoleFull)
 for   PR 2087 at commit 
[`305ad9f`](https://github.com/apache/spark/commit/305ad9f50b4dae992e09fdc962ebaf71e191a191).
 * This patch **passes** 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3366][MLLIB]Compute best splits distrib...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2595#issuecomment-57425687
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21093/consoleFull)
 for   PR 2595 at commit 
[`f13b346`](https://github.com/apache/spark/commit/f13b3461efb90e81492ef734c1534950f470641e).
 * This patch **fails** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class PStatsParam(AccumulatorParam):`



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: SPARK-2621. Update task InputMetrics increment...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3366][MLLIB]Compute best splits distrib...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService

2014-10-01 Thread aarondav
Github user aarondav commented on a diff in the pull request:

https://github.com/apache/spark/pull/2330#discussion_r18263553
  
--- Diff: 
core/src/main/scala/org/apache/spark/network/BlockDataManager.scala ---
@@ -20,14 +20,14 @@ package org.apache.spark.network
 import org.apache.spark.storage.StorageLevel
 
 
+private[spark]
 trait BlockDataManager {
 
   /**
-   * Interface to get local block data.
-   *
-   * @return Some(buffer) if the block exists locally, and None if it 
doesn't.
+   * Interface to get local block data. Throws an exception if the block 
cannot be found or
+   * cannot be read successfully.
*/
-  def getBlockData(blockId: String): Option[ManagedBuffer]
+  def getBlockData(blockId: String): ManagedBuffer
--- End diff --

Not sure how I feel about this taking a blockId as a string, especially if 
it's implemented by the BlockManager itself. What's the reasoning behind not 
taking a BlockId? Just too many users of the API that only have a String?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3343] [SQL] Add serde support for CTAS

2014-10-01 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/2570#discussion_r18263559
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
 ---
@@ -30,25 +32,38 @@ import org.apache.spark.sql.hive.MetastoreRelation
  * Create table and insert the query result into it.
  * @param database the database name of the new relation
  * @param tableName the table name of the new relation
- * @param insertIntoRelation function of creating the 
`InsertIntoHiveTable` 
- *by specifying the `MetaStoreRelation`, the data will be inserted 
into that table.
- * TODO Add more table creating properties,  e.g. SerDe, StorageHandler, 
in-memory cache etc.
+ * @param allowExisting allow continue working if it's already exists, 
otherwise
+ *  raise exception
+ * @param extra the extra information for this Operator, it should be the
+ *  ASTNode object for extracting the CreateTableDesc.
+ * @param query the query whose result will be insert into the new relation
  */
 @Experimental
 case class CreateTableAsSelect(
   database: String,
   tableName: String,
-  query: SparkPlan,
-  insertIntoRelation: MetastoreRelation = InsertIntoHiveTable)
+  allowExisting: Boolean,
+  extra: AnyRef,
+  query: LogicalPlan)
 extends LeafNode with Command {
 
   def output = Seq.empty
 
+  private[this] def sc = sqlContext.asInstanceOf[HiveContext]
+
   // A lazy computing of the metastoreRelation
   private[this] lazy val metastoreRelation: MetastoreRelation = {
-// Create the table 
-val sc = sqlContext.asInstanceOf[HiveContext]
-sc.catalog.createTable(database, tableName, query.output, false)
+// Get the CreateTableDesc from Hive SemanticAnalyzer
+val sa = new SemanticAnalyzer(sc.hiveconf)
--- End diff --

I was planning to do this by re-implementing within `HiveQL`, but the logic 
of `CreateTableDesc` is quite complicated and error-prone, probably reuse the 
Hive `SemanticAnalzyer` is a simple and quick work around. I will add a TODO 
for further improvement. What do you think?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService

2014-10-01 Thread aarondav
Github user aarondav commented on a diff in the pull request:

https://github.com/apache/spark/pull/2330#discussion_r18263556
  
--- Diff: 
core/src/main/scala/org/apache/spark/network/BlockTransferService.scala ---
@@ -83,7 +84,7 @@ abstract class BlockTransferService {
 val lock = new Object
 @volatile var result: Either[ManagedBuffer, Throwable] = null
--- End diff --

Out of curiosity, is there an advantage of this over `Try[ManagedBuffer]`?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3366][MLLIB]Compute best splits distrib...

2014-10-01 Thread Ishiihara
Github user Ishiihara commented on the pull request:

https://github.com/apache/spark/pull/2595#issuecomment-57426079
  
@chouqin You can run SPARK_TESTING=1 ./bin/pyspark 
python/pyspark/my_file.py to run unit tests for a certain file. In your case, 
use SPARK_TESTING=1 ./bin/pyspark python/pyspark/mllib/tree.py


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3751] [mllib] DecisionTree: example upd...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2604#issuecomment-57426168
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21092/consoleFull)
 for   PR 2604 at commit 
[`07b1fae`](https://github.com/apache/spark/commit/07b1fae6d8dc51abf44db0d0260383f1a3885761).
 * This patch **passes** 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3751] [mllib] DecisionTree: example upd...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3751] [mllib] DecisionTree: example upd...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2604#issuecomment-57426315
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21094/consoleFull)
 for   PR 2604 at commit 
[`b2b3c60`](https://github.com/apache/spark/commit/b2b3c600f8b66d2c877047faf9e70379339f4d45).
 * This patch **passes** 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3751] [mllib] DecisionTree: example upd...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3707] [SQL] Fix bug of type coercion in...

2014-10-01 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/2559#discussion_r18263943
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
 ---
@@ -74,7 +81,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter {
 val e = intercept[RuntimeException] {
   caseSensitiveAnalyze(UnresolvedRelation(None, tAbLe, None))
 }
-assert(e.getMessage === Table Not Found: tAbLe)
+assert(e.getMessage == Table Not Found: tAbLe)
--- End diff --

Thanks for noticing this, I have reverted 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService

2014-10-01 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/2330#discussion_r18263961
  
--- Diff: 
core/src/main/scala/org/apache/spark/network/BlockDataManager.scala ---
@@ -20,14 +20,14 @@ package org.apache.spark.network
 import org.apache.spark.storage.StorageLevel
 
 
+private[spark]
 trait BlockDataManager {
 
   /**
-   * Interface to get local block data.
-   *
-   * @return Some(buffer) if the block exists locally, and None if it 
doesn't.
+   * Interface to get local block data. Throws an exception if the block 
cannot be found or
+   * cannot be read successfully.
*/
-  def getBlockData(blockId: String): Option[ManagedBuffer]
+  def getBlockData(blockId: String): ManagedBuffer
--- End diff --

see the pr description on the todos as separate prs


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3707] [SQL] Fix bug of type coercion in...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2559#issuecomment-57427168
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21096/consoleFull)
 for   PR 2559 at commit 
[`d1c553f`](https://github.com/apache/spark/commit/d1c553f49780ec5933c7b5880298f0c1156d9bae).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2538#issuecomment-57427175
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21097/consoleFull)
 for   PR 2538 at commit 
[`6f0da2f`](https://github.com/apache/spark/commit/6f0da2fa486c2a580045a2e9e3133b6617875363).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService

2014-10-01 Thread aarondav
Github user aarondav commented on a diff in the pull request:

https://github.com/apache/spark/pull/2330#discussion_r18264123
  
--- Diff: core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala 
---
@@ -34,11 +35,17 @@ import org.apache.spark.util.{ByteBufferInputStream, 
Utils}
  * This interface provides an immutable view for data in the form of 
bytes. The implementation
  * should specify how the data is provided:
  *
- * - FileSegmentManagedBuffer: data backed by part of a file
- * - NioByteBufferManagedBuffer: data backed by a NIO ByteBuffer
- * - NettyByteBufManagedBuffer: data backed by a Netty ByteBuf
+ * - [[FileSegmentManagedBuffer]]: data backed by part of a file
+ * - [[NioManagedBuffer]]: data backed by a NIO ByteBuffer
+ * - [[NettyManagedBuffer]]: data backed by a Netty ByteBuf
+ *
+ * The concrete buffer implementation might be managed outside the JVM 
garbage collector.
+ * For example, in the case of [[NettyManagedBuffer]], the buffers are 
reference counted.
+ * In that case, if the buffer is going to be passed around to a different 
thread, retain/release
--- End diff --

The comment here suggests that you should only have to retain if your 
buffer is a NettyManagedBuffer, which isn't quite true, as you could have an 
NioManagedBuffer whose ByteBuffer underlies a Netty ByteBuf, in which case it 
is never safe to pass around.

I'm just a little worried about sanitary buffer usage, as misuse of this 
API from not copying could lead to nondeterministic data corruption.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2538#issuecomment-57427248
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21097/consoleFull)
 for   PR 2538 at commit 
[`6f0da2f`](https://github.com/apache/spark/commit/6f0da2fa486c2a580045a2e9e3133b6617875363).
 * This patch **fails** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class StreamingContext(object):`
  * `class DStream(object):`
  * `class TransformedDStream(DStream):`
  * `class RDDFunction(object):`
  * `class RDDFunctionSerializer(object):`
  * `abstract class PythonDStream(parent: DStream[_], @transient pfunc: 
PythonRDDFunction)`
  * `class PythonTransformedDStream (parent: DStream[_], @transient pfunc: 
PythonRDDFunction,`
  * `class PythonTransformed2DStream(parent: DStream[_], parent2: 
DStream[_],`
  * `class PythonStateDStream(parent: DStream[Array[Byte]], @transient 
reduceFunc: PythonRDDFunction)`
  * `class PythonReducedWindowedDStream(parent: DStream[Array[Byte]],`



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3707] [SQL] Fix bug of type coercion in...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3707] [SQL] Fix bug of type coercion in...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2559#issuecomment-57427564
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21096/consoleFull)
 for   PR 2559 at commit 
[`d1c553f`](https://github.com/apache/spark/commit/d1c553f49780ec5933c7b5880298f0c1156d9bae).
 * This patch **fails** 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3747] TaskResultGetter could incorrectl...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2599#issuecomment-57428215
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/220/consoleFull)
 for   PR 2599 at commit 
[`0e8d44c`](https://github.com/apache/spark/commit/0e8d44ccd930a655b85b184889c965d2c2e68694).
 * This patch **passes** 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3747] TaskResultGetter could incorrectl...

2014-10-01 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3747] TaskResultGetter could incorrectl...

2014-10-01 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/2599#issuecomment-57428433
  
Merging in master  branch-1.1.


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

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



[GitHub] spark pull request: SPARK-1830 Deploy failover, Make Persistence e...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/771#issuecomment-57428783
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21098/consoleFull)
 for   PR 771 at commit 
[`022cb8e`](https://github.com/apache/spark/commit/022cb8e531b2f7ffbda117cd41ac8f82cc5118fe).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-3747] TaskResultGetter could incorrectl...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3747] TaskResultGetter could incorrectl...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2599#issuecomment-57429092
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21095/consoleFull)
 for   PR 2599 at commit 
[`a74c04d`](https://github.com/apache/spark/commit/a74c04d5770b9d120f71df91563251f680ae37d3).
 * This patch **passes** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class PStatsParam(AccumulatorParam):`



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3755][Core] Do not bind port 1 - 1024 t...

2014-10-01 Thread scwf
GitHub user scwf opened a pull request:

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

[SPARK-3755][Core] Do not bind port 1 - 1024 to server in spark

Non-root user use port 1- 1024 to start jetty server will get the exception 
 java.net.SocketException: Permission denied, so not use these port 


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

$ git pull https://github.com/scwf/spark 1-1024

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

https://github.com/apache/spark/pull/2610.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 #2610


commit cb8cc76ef7548c24a7152dcfa799354f347a3a9f
Author: scwf wangf...@huawei.com
Date:   2014-10-01T07:32:24Z

do not use port 1 - 1024




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3755][Core] Do not bind port 1 - 1024 t...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/2610#issuecomment-57429408
  
Can one of the admins verify this patch?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3748] Log thread name in unit test logs

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2600#issuecomment-57429947
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/221/consoleFull)
 for   PR 2600 at commit 
[`83ffe88`](https://github.com/apache/spark/commit/83ffe888ce7cf230b4b1721bbf1de7bdfffa7c2c).
 * This patch **passes** 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2458] Make failed application log visib...

2014-10-01 Thread tsudukim
Github user tsudukim commented on the pull request:

https://github.com/apache/spark/pull/1558#issuecomment-57430441
  
Thank you @andrewor14
I've researched this problem these days with our environment and it turned 
out to be a very rare case as @vanzin suggested first.
 (like jvm lost and failed to call SparkContext::stop(), failed to write to 
HDFS for some reason, etc)
And my PR is not the smart way to solve the rare case.
so I drop this PR.
Thank you for your comments again.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2458] Make failed application log visib...

2014-10-01 Thread tsudukim
Github user tsudukim closed the pull request at:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3751] [mllib] DecisionTree: example upd...

2014-10-01 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/2604#issuecomment-57431357
  
LGTM. Merged into master. 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3748] Log thread name in unit test logs

2014-10-01 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/2600#issuecomment-57431314
  
Merging in 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3748] Log thread name in unit test logs

2014-10-01 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3751] [mllib] DecisionTree: example upd...

2014-10-01 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3756] check exception is caused by an a...

2014-10-01 Thread scwf
GitHub user scwf opened a pull request:

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

[SPARK-3756] check exception is caused by an address-port collision properly

Jetty server use MultiException to handle exceptions when start server
refer 
https://github.com/eclipse/jetty.project/blob/jetty-8.1.14.v20131031/jetty-server/src/main/java/org/eclipse/jetty/server/Server.java

So in ```isBindCollision``` add the logical to cover MultiException

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

$ git pull https://github.com/scwf/spark fix-isBindCollision

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

https://github.com/apache/spark/pull/2611.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 #2611


commit 3a6c8492baf8da5b49a3ced99dfc7378d13f7816
Author: scwf wangf...@huawei.com
Date:   2014-10-01T08:03:40Z

fix bug in isBindCollision




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3756] [Core]check exception is caused b...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/2611#issuecomment-57432064
  
Can one of the admins verify this patch?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3343] [SQL] Add serde support for CTAS

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [WIP][SPARK-1405][MLLIB] topic modeling on Gra...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2388#issuecomment-57433219
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21100/consoleFull)
 for   PR 2388 at commit 
[`e00f5a6`](https://github.com/apache/spark/commit/e00f5a6352a2bbc7d8c5cdd87079f1d55a0b910a).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [WIP][SPARK-1405][MLLIB] topic modeling on Gra...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [WIP][SPARK-1405][MLLIB] topic modeling on Gra...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2388#issuecomment-57433323
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21100/consoleFull)
 for   PR 2388 at commit 
[`e00f5a6`](https://github.com/apache/spark/commit/e00f5a6352a2bbc7d8c5cdd87079f1d55a0b910a).
 * This patch **fails** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class TopicModelingKryoRegistrator extends KryoRegistrator `



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [WIP][SPARK-1405][MLLIB] topic modeling on Gra...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2388#issuecomment-57433700
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21101/consoleFull)
 for   PR 2388 at commit 
[`84f51e3`](https://github.com/apache/spark/commit/84f51e3857f6ffae0584100f53ac7e68767ba060).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: SPARK-1830 Deploy failover, Make Persistence e...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/771#issuecomment-57434816
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21098/consoleFull)
 for   PR 771 at commit 
[`022cb8e`](https://github.com/apache/spark/commit/022cb8e531b2f7ffbda117cd41ac8f82cc5118fe).
 * This patch **passes** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class ApplicationInfo(`
  * `class DriverInfo(`
  * `trait LeaderElectionAgent `
  * `trait LeaderElectable `
  * `trait PersistenceEngine `
  * `abstract class StandaloneRecoveryModeFactory(conf: SparkConf) `
  * `class WorkerInfo(`



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: SPARK-1830 Deploy failover, Make Persistence e...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3366][MLLIB]Compute best splits distrib...

2014-10-01 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/2595#issuecomment-57434938
  
@chouqin The trained model only contains a single node in the python test. 
Maybe there is a bug that caused early termination.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread kmader
Github user kmader commented on a diff in the pull request:

https://github.com/apache/spark/pull/1658#discussion_r18267344
  
--- Diff: core/src/main/scala/org/apache/spark/input/RawFileInput.scala ---
@@ -0,0 +1,221 @@
+/*
+ * 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.input
+
+import scala.collection.JavaConversions._
+import com.google.common.io.{ByteStreams, Closeables}
+import org.apache.hadoop.mapreduce.InputSplit
+import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit
+import org.apache.hadoop.mapreduce.RecordReader
+import org.apache.hadoop.mapreduce.TaskAttemptContext
+import org.apache.hadoop.fs.{FSDataInputStream, Path}
+import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat
+import org.apache.hadoop.mapreduce.JobContext
+import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader
+import java.io.DataInputStream
+
+
+/**
+ *  A general format for reading whole files in as streams, byte arrays,
+ *  or other functions to be added
+ */
+abstract class StreamFileInputFormat[T]
+  extends CombineFileInputFormat[String,T]  {
+  override protected def isSplitable(context: JobContext, file: Path): 
Boolean = false
+  /**
+   * Allow minPartitions set by end-user in order to keep compatibility 
with old Hadoop API.
+   */
+  def setMaxSplitSize(context: JobContext, minPartitions: Int) {
+val files = listStatus(context)
+val totalLen = files.map { file =
+  if (file.isDir) 0L else file.getLen
+}.sum
+
+val maxSplitSize = Math.ceil(totalLen*1.0/files.length).toLong
+super.setMaxSplitSize(maxSplitSize)
+  }
+
+  def createRecordReader(split: InputSplit, taContext: TaskAttemptContext):
+  RecordReader[String,T]
+
+}
+
+/**
+ * A class that allows DataStreams to be serialized and moved around by 
not creating them
+ * until they need to be read
+ */
+class PortableDataStream(split: CombineFileSplit, context: 
TaskAttemptContext, index: Integer)
+  extends Serializable {
+
+  private var fileIn: FSDataInputStream = 
null.asInstanceOf[FSDataInputStream]
+  private var isOpen = false
+  /**
+   * Calculate the path name independently of opening the file
+   */
+  private lazy val path = {
+val pathp = split.getPath(index)
+pathp.toString
+  }
+
+  /**
+   * create a new DataInputStream from the split and context
+   */
+  def open(): FSDataInputStream = {
+val pathp = split.getPath(index)
+val fs = pathp.getFileSystem(context.getConfiguration)
+fileIn = fs.open(pathp)
+isOpen=true
+fileIn
+  }
+
+  /**
+   * close the file (if it is already open)
+   */
+  def close() = {
+if (isOpen) {
+  try {
+fileIn.close()
+isOpen=false
+  } catch {
+case ioe: java.io.IOException = // do nothing
+  }
+}
+  }
+  def getPath(): String = path
+}
+
+/**
+ * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader 
RecordReader]]
+ * to reading files out as streams
+ */
+abstract class StreamBasedRecordReader[T](
+   split: CombineFileSplit,
+   context: TaskAttemptContext,
+   index: Integer)
+  extends RecordReader[String, T] {
+
+
+
+  // True means the current file has been processed, then skip it.
+  private var processed = false
+
+  private var key = 
+  private var value: T = null.asInstanceOf[T]
+
+
+  override def initialize(split: InputSplit, context: TaskAttemptContext) 
= {}
+  override def close() = {}
+
+  override def getProgress = if (processed) 1.0f else 0.0f
+
+  override def getCurrentKey = key
+
+  override def getCurrentValue = value
+
+
+
+  override 

[GitHub] spark pull request: [SPARK-3756] [Core]check exception is caused b...

2014-10-01 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/2611#discussion_r18267520
  
--- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala ---
@@ -1470,6 +1472,7 @@ private[spark] object Utils extends Logging {
   return true
 }
 isBindCollision(e.getCause)
+  case e: MultiException = e.getThrowables.map(ex = 
isBindCollision(ex)).reduceLeft(_ || _)
--- End diff --

I think you can just write `e.getThrowables.exists(isBindCollision)`?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread kmader
Github user kmader commented on a diff in the pull request:

https://github.com/apache/spark/pull/1658#discussion_r18267674
  
--- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
@@ -511,6 +511,67 @@ class SparkContext(config: SparkConf) extends Logging {
   }
 
   /**
+   * Get an RDD for a Hadoop-readable dataset as byte-streams for each file
+   * (useful for binary data)
+   *
+   * @param minPartitions A suggestion value of the minimal splitting 
number for input data.
+   *
+   * @note Small files are preferred, large file is also allowable, but 
may cause bad performance.
+   */
+  def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions):
+  RDD[(String, Array[Byte])] = {
+val job = new NewHadoopJob(hadoopConfiguration)
+NewFileInputFormat.addInputPath(job, new Path(path))
+val updateConf = job.getConfiguration
+new BinaryFileRDD(
+  this,
+  classOf[ByteInputFormat],
+  classOf[String],
+  classOf[Array[Byte]],
+  updateConf,
+  minPartitions).setName(path)
+  }
+
+  /**
+   * Get an RDD for a Hadoop-readable dataset as PortableDataStream for 
each file
+   * (useful for binary data)
+   *
+   *
+   * @param minPartitions A suggestion value of the minimal splitting 
number for input data.
+   *
+   * @note Care must be taken to close the files afterwards
+   * @note Small files are preferred, large file is also allowable, but 
may cause bad performance.
+   */
+  @DeveloperApi
+  def dataStreamFiles(path: String, minPartitions: Int = 
defaultMinPartitions):
+  RDD[(String, PortableDataStream)] = {
+val job = new NewHadoopJob(hadoopConfiguration)
+NewFileInputFormat.addInputPath(job, new Path(path))
+val updateConf = job.getConfiguration
+new BinaryFileRDD(
+  this,
+  classOf[StreamInputFormat],
+  classOf[String],
+  classOf[PortableDataStream],
+  updateConf,
+  minPartitions).setName(path)
+  }
+
+  /**
+   * Load data from a flat binary file, assuming each record is a set of 
numbers
+   * with the specified numerical format (see ByteBuffer), and the number 
of
+   * bytes per record is constant (see FixedLengthBinaryInputFormat)
+   *
+   * @param path Directory to the input data files
+   * @return An RDD of data with values, RDD[(Array[Byte])]
+   */
+  def binaryRecords(path: String): RDD[Array[Byte]] = {
--- End diff --

Yes this makes much more sense, I had just copied the code from 
@freeman-lab, but I made it into a parameter 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3756] [Core]check exception is caused b...

2014-10-01 Thread scwf
Github user scwf commented on a diff in the pull request:

https://github.com/apache/spark/pull/2611#discussion_r18268153
  
--- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala ---
@@ -1470,6 +1472,7 @@ private[spark] object Utils extends Logging {
   return true
 }
 isBindCollision(e.getCause)
+  case e: MultiException = e.getThrowables.map(ex = 
isBindCollision(ex)).reduceLeft(_ || _)
--- End diff --

yes, thanks, more cleanly


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

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



[GitHub] spark pull request: [SPARK-3758] [Windows] Wrong EOL character in ...

2014-10-01 Thread sarutak
GitHub user sarutak opened a pull request:

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

[SPARK-3758] [Windows] Wrong EOL character in *.cmd

Windows platform expects the EOL characters are CRLF but in *.cmd except 
for compute-classpath.cmd, the EOL characters are LF.
To avoid unexpected problem, we should replace LF with CRLF in *.cmd.

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

$ git pull https://github.com/sarutak/spark SPARK-3758

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

https://github.com/apache/spark/pull/2612.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 #2612


commit 1fe76a1e3720d73623a357c6068646009192a7f7
Author: Kousuke Saruta saru...@oss.nttdata.co.jp
Date:   2014-10-01T09:13:06Z

Modified EOL character from LF to CRLF in *.cmd




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3758] [Windows] Wrong EOL character in ...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2612#issuecomment-57439114
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21102/consoleFull)
 for   PR 2612 at commit 
[`1fe76a1`](https://github.com/apache/spark/commit/1fe76a1e3720d73623a357c6068646009192a7f7).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-1892][MLLIB] Adding OWL-QN optimizer fo...

2014-10-01 Thread dbtsai
Github user dbtsai commented on the pull request:

https://github.com/apache/spark/pull/840#issuecomment-57439459
  
@debasish83 and @codedeft The weighted method for OWLQN in breeze is merged 
https://github.com/scalanlp/breeze/commit/2570911026aa05aa1908ccf7370bc19cd8808a4c

I will submit a PR to Spark to use newer version of breeze with this 
feature once @dlwh publishes to this to maven. But there is still some work in 
mllib side to have it working properly. I'll work on this once I'm back from 
vacation.  



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [WIP][SPARK-1405][MLLIB] topic modeling on Gra...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2388#issuecomment-57440229
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21101/consoleFull)
 for   PR 2388 at commit 
[`84f51e3`](https://github.com/apache/spark/commit/84f51e3857f6ffae0584100f53ac7e68767ba060).
 * This patch **passes** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class TopicModelingKryoRegistrator extends KryoRegistrator `



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [WIP][SPARK-1405][MLLIB] topic modeling on Gra...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/1658#issuecomment-57440499
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21103/consoleFull)
 for   PR 1658 at commit 
[`238c83c`](https://github.com/apache/spark/commit/238c83cc9eeab7012aad1a3e2660aae31073a56d).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/1658#issuecomment-57440605
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21103/consoleFull)
 for   PR 1658 at commit 
[`238c83c`](https://github.com/apache/spark/commit/238c83cc9eeab7012aad1a3e2660aae31073a56d).
 * This patch **fails** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class PortableDataStream(split: CombineFileSplit, context: 
TaskAttemptContext, index: Integer)`
  * `abstract class StreamBasedRecordReader[T](`
  * `abstract class BinaryRecordReader[T](`



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3757] mvn clean doesn't delete some fil...

2014-10-01 Thread tsudukim
GitHub user tsudukim opened a pull request:

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

[SPARK-3757] mvn clean doesn't delete some files

Added directory to be deleted into maven-clean-plugin in pom.xml.

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

$ git pull https://github.com/tsudukim/spark feature/SPARK-3757

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

https://github.com/apache/spark/pull/2613.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 #2613


commit 67c7171e277d099e87972302ee798a310c0da2e6
Author: Masayoshi TSUZUKI tsudu...@oss.nttdata.co.jp
Date:   2014-10-01T09:09:56Z

[SPARK-3757] mvn clean doesn't delete some files

Added directory to be deleted into maven-clean-plugin.

commit 8804bfc9f4cc7fe4f803f3145c1fa7f5bc902d70
Author: Masayoshi TSUZUKI tsudu...@oss.nttdata.co.jp
Date:   2014-10-01T09:39:32Z

Modified indent.




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3757] mvn clean doesn't delete some fil...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/2613#issuecomment-57440787
  
Can one of the admins verify this patch?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3757] mvn clean doesn't delete some fil...

2014-10-01 Thread sarutak
Github user sarutak commented on the pull request:

https://github.com/apache/spark/pull/2613#issuecomment-57441202
  
Jenkins, test 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: Typo error in KafkaWordCount example

2014-10-01 Thread gasparms
GitHub user gasparms opened a pull request:

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

Typo error in KafkaWordCount example

topicpMap to topicMap

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

$ git pull https://github.com/gasparms/spark patch-1

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

https://github.com/apache/spark/pull/2614.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 #2614


commit 00aab2c5a66f3d01e2b9f5291c6bdff656fafc28
Author: Gaspar Munoz munozs...@gmail.com
Date:   2014-10-01T09:50:29Z

Typo error in KafkaWordCount example

topicpMap to topicMap




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: Typo error in KafkaWordCount example

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/2614#issuecomment-57441727
  
Can one of the admins verify this patch?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3758] [Windows] Wrong EOL character in ...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3758] [Windows] Wrong EOL character in ...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2612#issuecomment-57445492
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21102/consoleFull)
 for   PR 2612 at commit 
[`1fe76a1`](https://github.com/apache/spark/commit/1fe76a1e3720d73623a357c6068646009192a7f7).
 * This patch **passes** 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: Adjust build system and tests to work with sca...

2014-10-01 Thread ScrapCodes
GitHub user ScrapCodes opened a pull request:

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

Adjust build system and tests to work with scala 2.11+ repl port.



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

$ git pull https://github.com/ScrapCodes/spark-1 scala-2.11-full

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

https://github.com/apache/spark/pull/2615.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 #2615


commit b9389dba44cbc7c68cc9dcadd74ed10ec9a648c9
Author: Prashant Sharma prashan...@imaginea.com
Date:   2014-08-12T10:55:57Z

Adjust build system and tests to work with scala 2.11




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

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



[GitHub] spark pull request: Adjust build system and tests to work with sca...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2615#issuecomment-57447009
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21104/consoleFull)
 for   PR 2615 at commit 
[`b9389db`](https://github.com/apache/spark/commit/b9389dba44cbc7c68cc9dcadd74ed10ec9a648c9).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: Adjust build system and tests to work with sca...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2615#issuecomment-57447144
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21104/consoleFull)
 for   PR 2615 at commit 
[`b9389db`](https://github.com/apache/spark/commit/b9389dba44cbc7c68cc9dcadd74ed10ec9a648c9).
 * This patch **fails** 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: Adjust build system and tests to work with sca...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [ SPARK-1812] Adjust build system and tests to...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2615#issuecomment-57447849
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21106/consoleFull)
 for   PR 2615 at commit 
[`83998bf`](https://github.com/apache/spark/commit/83998bfc095d85db452e189d57046096f1e783ec).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [ SPARK-1812] Adjust build system and tests to...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: SPARK-2548 [STREAMING] JavaRecoverableWordCoun...

2014-10-01 Thread srowen
Github user srowen commented on the pull request:

https://github.com/apache/spark/pull/2564#issuecomment-57449006
  
@tdas No problem, text removed. I tested the Java example using the 
instructions in the javadoc, and that worked. I was lazy, and didn't try it on 
a cluster and try killing the receiver and recovering, on the assumption that 
the API calls are correct and it's the tests that should make sure it works. 
It's a straight port of the Scala example, so if that is a valid example, this 
should be too.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: SPARK-2548 [STREAMING] JavaRecoverableWordCoun...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2564#issuecomment-57449117
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21107/consoleFull)
 for   PR 2564 at commit 
[`35f23e3`](https://github.com/apache/spark/commit/35f23e36031635e48934573ace0aa5cf22c26bd6).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/1658#issuecomment-57451347
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21108/consoleFull)
 for   PR 1658 at commit 
[`19812a8`](https://github.com/apache/spark/commit/19812a83df8a4852412feb7dec7f42126b0b139e).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/1658#issuecomment-57451425
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21108/consoleFull)
 for   PR 1658 at commit 
[`19812a8`](https://github.com/apache/spark/commit/19812a83df8a4852412feb7dec7f42126b0b139e).
 * This patch **fails** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class PortableDataStream(@transient isplit: CombineFileSplit, 
@transient context: TaskAttemptContext, index: Integer)`
  * `abstract class StreamBasedRecordReader[T](`
  * `abstract class BinaryRecordReader[T](`



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/1658#issuecomment-57452596
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21109/consoleFull)
 for   PR 1658 at commit 
[`4163e38`](https://github.com/apache/spark/commit/4163e38bccca33608fc4a241760e86d4862793b5).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: SPARK-2548 [STREAMING] JavaRecoverableWordCoun...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2564#issuecomment-57455207
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21107/consoleFull)
 for   PR 2564 at commit 
[`35f23e3`](https://github.com/apache/spark/commit/35f23e36031635e48934573ace0aa5cf22c26bd6).
 * This patch **passes** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `public final class JavaRecoverableNetworkWordCount `



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: SPARK-2548 [STREAMING] JavaRecoverableWordCoun...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/1658#issuecomment-57457518
  
  [QA tests have 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21109/consoleFull)
 for   PR 1658 at commit 
[`4163e38`](https://github.com/apache/spark/commit/4163e38bccca33608fc4a241760e86d4862793b5).
 * This patch **fails** unit tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class PortableDataStream(@transient isplit: CombineFileSplit,`
  * `abstract class StreamBasedRecordReader[T](`
  * `abstract class BinaryRecordReader[T](`



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/1658#issuecomment-57458416
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21110/consoleFull)
 for   PR 1658 at commit 
[`0588737`](https://github.com/apache/spark/commit/05887379eafdc359206753a68571aaf3fb2dd7a6).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [ SPARK-1812] Adjust build system and tests to...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2615#issuecomment-57458979
  
**[Tests timed 
out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21106/consoleFull)**
 for PR 2615 at commit 
[`83998bf`](https://github.com/apache/spark/commit/83998bfc095d85db452e189d57046096f1e783ec)
 after a configured wait of `120m`.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [ SPARK-1812] Adjust build system and tests to...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3007][SQL] WIP: adds dynamic partitioni...

2014-10-01 Thread liancheng
GitHub user liancheng opened a pull request:

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

[SPARK-3007][SQL] WIP: adds dynamic partitioning support

PR #2226 was reverted because of it broke Jenkins builds for unknown 
reason. This debugging PR aims to fix the Jenkins build.

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

$ git pull https://github.com/liancheng/spark dp-fix

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

https://github.com/apache/spark/pull/2616.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 #2616


commit 6bb58808875e82740da64bbb7cbca4d22817dbce
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-12T17:27:54Z

Update HiveQl.scala

commit 1867e23f72f94c7161a485222c6a8e31814d20f4
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-12T17:29:36Z

Update SparkHadoopWriter.scala

commit adf02f13e8d93eadb2d032007e00cd9406c62439
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-12T17:31:01Z

Update InsertIntoHiveTable.scala

commit 6af73f46430c5e38d43af8ed288936e1b4ca2678
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-12T17:53:04Z

Update InsertIntoHiveTable.scala

commit 98cfb1fd8dab201f9411dd487e4a8b259216d9a3
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-18T06:29:21Z

Update HiveCompatibilitySuite.scala

commit 37c603b5001169d1aa21fa293f782666eedc338a
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-18T06:34:53Z

Update InsertIntoHiveTable.scala

commit d452eb322da5962875c8e96865f1cd63b7a0
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-18T06:36:58Z

Update HiveQuerySuite.scala

commit 051ba91e4952c463446450c745073ab4cf742b26
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-18T07:18:07Z

Update Cast.scala

commit 8ad173cd41e2abb93df681383cf15aa59e3484c5
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-18T07:44:07Z

Update InsertIntoHiveTable.scala

commit 3f91665c1e546881f045034023cace851e1949d4
Author: baishuo(白硕) vc_j...@hotmail.com
Date:   2014-08-18T07:57:20Z

Update Cast.scala

commit 8e7268cae0f4f9463c092b6c2833a86be56524e7
Author: baishuo vc_j...@hotmail.com
Date:   2014-08-19T16:01:49Z

update file after test

commit cd822f0471ddf51635e0c2a7054725ab155f5939
Author: baishuo vc_j...@hotmail.com
Date:   2014-08-19T17:14:53Z

do a little modify

commit b660e74574585a0572b5b21aba1ee42a87d89437
Author: baishuo vc_j...@hotmail.com
Date:   2014-09-03T03:28:29Z

delete a empty else branch

commit caea6fb7db2a34e6e0949aabb843b7f728cab541
Author: baishuo vc_j...@hotmail.com
Date:   2014-09-03T06:03:34Z

modify code to pass scala style checks

commit 207c6ace8774f6a89411e0cd929eaedb9f957a8d
Author: baishuo vc_j...@hotmail.com
Date:   2014-09-03T08:22:19Z

modify for some bad indentation

commit 761ecf20365f2864e28b7cad2c0550b159724eb3
Author: baishuo vc_j...@hotmail.com
Date:   2014-09-09T09:57:48Z

modify according micheal's advice

commit 997c990855d94dd2df2ab412566c438def97de61
Author: baishuo vc_j...@hotmail.com
Date:   2014-09-10T02:51:20Z

use HiveConf.DEFAULTPARTITIONNAME to replace 
hive.exec.default.partition.name

commit b8216119605132bb70a8862e921b0861d5ec7f9f
Author: baishuo vc_j...@hotmail.com
Date:   2014-09-10T03:09:20Z

pass check style

commit d53daa5a263cf0b0fdf47111f1d8ba0c55d08b24
Author: Cheng Lian lian.cs@gmail.com
Date:   2014-09-16T07:06:25Z

Refactors dynamic partitioning support

Conflicts:

sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala

commit 6fb16d76089e07c4fc273a86c7f887949e904b57
Author: Cheng Lian lian.cs@gmail.com
Date:   2014-09-17T01:32:12Z

Fixes typo in test name, regenerated golden answer files

commit c47470e4660058b06f9a7704bad0920e69b246fe
Author: Cheng Lian lian.cs@gmail.com
Date:   2014-09-17T05:13:36Z

Refactors InsertIntoHiveTable to a Command

commit 922718165903bf12b3a2ee40241a60cb20d21886
Author: Cheng Lian lian.cs@gmail.com
Date:   2014-09-17T05:41:56Z

Minor refactoring

commit 26632c366ad6c8255b50d5e8a41bc23cddbd396b
Author: Cheng Lian lian.cs@gmail.com
Date:   2014-09-17T07:28:57Z

Adds more tests

commit 0eed349f5824ef3917af1e380bfb529f9875b0c1
Author: Cheng Lian lian.cs@gmail.com
Date:   2014-09-18T01:08:01Z

Addresses @yhuai's comments

commit 9c6eb2db8de06db0d800ebefb37fc016ecd4c88c
Author: Cheng Lian lian.cs@gmail.com
Date:   2014-09-22T04:49:31Z

Adds tests to verify dynamic partitioning folder layout

commit a132c800ecb785a0a5c36a969b824908eba2801c
Author: Cheng Lian lian.cs@gmail.com
Date:   2014-10-01T13:13:59Z

Fixes output compression




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

[GitHub] spark pull request: [SPARK-3343] [SQL] Add serde support for CTAS

2014-10-01 Thread chenghao-intel
Github user chenghao-intel commented on the pull request:

https://github.com/apache/spark/pull/2570#issuecomment-57461439
  
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-3343] [SQL] Add serde support for CTAS

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2570#issuecomment-57462077
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21112/consoleFull)
 for   PR 2570 at commit 
[`fcbbc61`](https://github.com/apache/spark/commit/fcbbc611d80a31160c79645a09970fba60559d48).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-3007][SQL] WIP: adds dynamic partitioni...

2014-10-01 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/2616#issuecomment-57462060
  
  [QA tests have 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/2/consoleFull)
 for   PR 2616 at commit 
[`a132c80`](https://github.com/apache/spark/commit/a132c800ecb785a0a5c36a969b824908eba2801c).
 * This patch merges cleanly.


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

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



[GitHub] spark pull request: [SPARK-2788] [STREAMING] Add location filterin...

2014-10-01 Thread sjbrunst
Github user sjbrunst commented on the pull request:

https://github.com/apache/spark/pull/1717#issuecomment-57464410
  
@tdas That would be a better long-term solution, as I'm sure there will be 
some users down the road who want to use the other parts of FilterQuery. That 
way they won't have to submit a PR like this every time they want to use a 
feature of FilterQuery that ```createStream``` doesn't have yet. I can 
implement that and push it to this PR.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-2759][CORE] Generic Binary File Support...

2014-10-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



  1   2   3   4   5   6   >