[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-27 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r80846949
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/SparkAesCipher.java
 ---
@@ -0,0 +1,270 @@
+/*
+ * 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.network.sasl.aes;
+
+import java.io.IOException;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.InvalidKeyException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.Properties;
+import javax.crypto.Cipher;
+import javax.crypto.Mac;
+import javax.crypto.SecretKey;
+import javax.crypto.ShortBufferException;
+import javax.crypto.spec.SecretKeySpec;
+import javax.crypto.spec.IvParameterSpec;
+import javax.security.sasl.SaslException;
+
+import org.apache.commons.crypto.cipher.CryptoCipher;
+import org.apache.commons.crypto.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * AES cipher for encryption and decryption.
+ */
+public class SparkAesCipher {
--- End diff --

Thanks for elaboration. I agree with you to implement AES encryption not 
SASL encryption. And here are reasons why AES encryption implemented like this:

First of all, to enable AES encryption it has to negotiate a pair of key 
and input vector through SASL, that's the connection between AES and SASL 
Encryption. 

Secondly, the patch is not intended to implement a SASL encryption but to 
implement the AES encryption by utilizing the existing encryption code provided 
by SASL encryption (the SaslEncryption class,  which wraps the 
SaslEncryptionBackend as netty channelhandlers). So the patch do implement 
netty channel handler while function names are wrap/unwrap against write.

Lastly, I think the code cannot be simpler if implement separated channel 
handlers, since AES encryption needs key negotiation, Integrity check(seq 
number and hmac) as I mentioned before.







---
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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/15263
  
Yeah, @HyukjinKwon . This PR is just to rewrite the implementation. No need 
to resolve all the comments here.


---
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 issue #15257: [SPARK-17683][SQL] Support ArrayType in Literal.apply

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15257
  
**[Test build #66023 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66023/consoleFull)**
 for PR 15257 at commit 
[`c9cdd29`](https://github.com/apache/spark/commit/c9cdd29424c27c600a9e22c0b8ddf1a5998b6194).


---
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 #14897: [SPARK-17338][SQL] add global temp view

2016-09-27 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14897#discussion_r80846005
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala ---
@@ -37,39 +37,14 @@ import org.apache.spark.util.{MutableURLClassLoader, 
Utils}
  */
 private[sql] class SharedState(val sparkContext: SparkContext) extends 
Logging {
 
-  /**
-   * Class for caching query results reused in future executions.
-   */
-  val cacheManager: CacheManager = new CacheManager
-
-  /**
-   * A listener for SQL-specific 
[[org.apache.spark.scheduler.SparkListenerEvent]]s.
-   */
-  val listener: SQLListener = createListenerAndUI(sparkContext)
-
+  // Load hive-site.xml into hadoopConf and determine the warehouse path 
we want to use, based on
+  // the config from both hive and Spark SQL. Finally set the warehouse 
config value to sparkConf.
   {
 val configFile = 
Utils.getContextOrSparkClassLoader.getResource("hive-site.xml")
 if (configFile != null) {
   sparkContext.hadoopConfiguration.addResource(configFile)
 }
-  }
-
-  /**
-   * A catalog that interacts with external systems.
-   */
-  lazy val externalCatalog: ExternalCatalog =
-SharedState.reflect[ExternalCatalog, SparkConf, Configuration](
-  SharedState.externalCatalogClassName(sparkContext.conf),
-  sparkContext.conf,
-  sparkContext.hadoopConfiguration)
-
-  /**
-   * A classloader used to load all user-added jar.
-   */
-  val jarClassLoader = new NonClosableMutableURLClassLoader(
-org.apache.spark.util.Utils.getContextOrSparkClassLoader)
 
-  {
 // Set the Hive metastore warehouse path to the one we use
 val tempConf = new SQLConf
--- End diff --

it's really bad that we put this initializing code in the middle of this 
class. This initializing code sets the warehouse path to sparkConf, which means 
it changes the state of the instance. If any fields that are declared before 
this initializing code and need to read this states, e.g. `externalCatalog`, 
they must be `lazy val`.

I think it's more reasonable to less error-prone to put this initializing 
code in the beginning of this class.


---
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 #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelatio...

2016-09-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15263#discussion_r80845890
  
--- Diff: docs/sql-programming-guide.md ---
@@ -1058,7 +1059,21 @@ the Data Sources API. The following options are 
supported:
   The JDBC fetch size, which determines how many rows to fetch per 
round trip. This can help performance on JDBC drivers which default to low 
fetch size (eg. Oracle with 10 rows).
 
   
-  
+
+  
+batchsize
+
+  The JDBC batch size, which determines how many rows to insert per 
round trip. This can help performance on JDBC drivers.
+
+  
+
+  
+isolationLevel
+
+  The transaction isolation level, which applies to current 
connection. Please refer the documenation in java.sql.Connection.
--- End diff --

`current connection` -> `the current connection`


---
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 #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelatio...

2016-09-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15263#discussion_r80845726
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
 ---
@@ -50,67 +51,48 @@ class JdbcRelationProvider extends 
CreatableRelationProvider
 JDBCRelation(jdbcOptions.url, jdbcOptions.table, parts, 
properties)(sqlContext.sparkSession)
   }
 
-  /*
-   * The following structure applies to this code:
-   * |tableExists|  !tableExists
-   
*
-   * Ignore  | BaseRelation  | CreateTable, saveTable, 
BaseRelation
-   * ErrorIfExists   | ERROR | CreateTable, saveTable, 
BaseRelation
-   * Overwrite*  | (DropTable, CreateTable,) | CreateTable, saveTable, 
BaseRelation
-   * | saveTable, BaseRelation   |
-   * Append  | saveTable, BaseRelation   | CreateTable, saveTable, 
BaseRelation
-   *
-   * *Overwrite & tableExists with truncate, will not drop & create, but 
instead truncate
-   */
   override def createRelation(
   sqlContext: SQLContext,
   mode: SaveMode,
   parameters: Map[String, String],
-  data: DataFrame): BaseRelation = {
-val jdbcOptions = new JDBCOptions(parameters)
-val url = jdbcOptions.url
-val table = jdbcOptions.table
-
+  df: DataFrame): BaseRelation = {
+val options = new JDBCOptions(parameters)
+val url = options.url
+val table = options.table
+val createTableOptions = options.createTableOptions
+val isTruncate = options.isTruncate
 val props = new Properties()
 props.putAll(parameters.asJava)
-val conn = JdbcUtils.createConnectionFactory(url, props)()
 
+val conn = JdbcUtils.createConnectionFactory(url, props)()
 try {
   val tableExists = JdbcUtils.tableExists(conn, url, table)
+  if (tableExists) {
+mode match {
+  case SaveMode.Overwrite =>
+if (isTruncate && 
isCascadingTruncateTable(url).contains(false)) {
+  // In this case, we should truncate table and then load.
+  truncateTable(conn, table)
+  saveTable(df, url, table, props)
+} else {
+  // Otherwise, do not truncate but just drop.
+  dropTable(conn, table)
+  createTable(df, url, table, createTableOptions, conn)
+  saveTable(df, url, table, props)
+}
+  case SaveMode.Append =>
+saveTable(df, url, table, props)
 
-  val (doCreate, doSave) = (mode, tableExists) match {
-case (SaveMode.Ignore, true) => (false, false)
-case (SaveMode.ErrorIfExists, true) => throw new AnalysisException(
-  s"Table or view '$table' already exists, and SaveMode is set to 
ErrorIfExists.")
-case (SaveMode.Overwrite, true) =>
-  if (jdbcOptions.isTruncate && 
JdbcUtils.isCascadingTruncateTable(url) == Some(false)) {
-JdbcUtils.truncateTable(conn, table)
-(false, true)
-  } else {
-JdbcUtils.dropTable(conn, table)
-(true, true)
-  }
-case (SaveMode.Append, true) => (false, true)
-case (_, true) => throw new IllegalArgumentException(s"Unexpected 
SaveMode, '$mode'," +
-  " for handling existing tables.")
-case (_, false) => (true, true)
-  }
+  case SaveMode.ErrorIfExists =>
+throw new AnalysisException(
+  s"Table or view '$table' already exists, and SaveMode is set 
to ErrorIfExists.")
--- End diff --

`Table or view '$table' already exists. SaveMode: ErrorIfExists.`


---
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 #15213: [SPARK-17644] [CORE] Do not add failedStages when...

2016-09-27 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/15213#discussion_r80845291
  
--- Diff: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
@@ -2105,6 +2107,59 @@ class DAGSchedulerSuite extends SparkFunSuite with 
LocalSparkContext with Timeou
 assert(scheduler.getShuffleDependencies(rddE) === Set(shuffleDepA, 
shuffleDepC))
   }
 
+  test("After one stage is aborted for too many failed attempts, 
subsequent stages" +
+"still behave correctly on fetch failures") {
+def fetchFailJob: Unit = {
+  val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 
1)).groupByKey()
+  val shuffleHandle =
+rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, 
_]].shuffleHandle
+  rdd1.map {
+case (x, _) if (x == 1) =>
+  throw new FetchFailedException(
+BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, 
"test")
+case (x, _) => x
+  }.count()
+}
+
+def successJob: Unit = {
+  object FailThisAttempt {
+val _fail = new AtomicBoolean(true)
+  }
+  val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 
1)).groupByKey()
+  val shuffleHandle =
+rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, 
_]].shuffleHandle
+  rdd1.map {
+case (x, _) if (x == 1) && FailThisAttempt._fail.getAndSet(false) 
=>
+  throw new FetchFailedException(
+BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, 
"test")
+  }
+}
+
+failAfter(60.seconds) {
+  val e = intercept[SparkException] {
+fetchFailJob
+  }
+  
assert(e.getMessage.contains("org.apache.spark.shuffle.FetchFailedException"))
+}
+
+// The following job that fails due to fetching failure will hang 
without
+// the fix for SPARK-17644
--- End diff --

Can you change to something like "Run a second job that will fail due to a 
fetch failure.  This job will hang without the fix for SPARK-17644."


---
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 #15213: [SPARK-17644] [CORE] Do not add failedStages when...

2016-09-27 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/15213#discussion_r80845128
  
--- Diff: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
@@ -2105,6 +2107,59 @@ class DAGSchedulerSuite extends SparkFunSuite with 
LocalSparkContext with Timeou
 assert(scheduler.getShuffleDependencies(rddE) === Set(shuffleDepA, 
shuffleDepC))
   }
 
+  test("After one stage is aborted for too many failed attempts, 
subsequent stages" +
+"still behave correctly on fetch failures") {
+def fetchFailJob: Unit = {
+  val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 
1)).groupByKey()
+  val shuffleHandle =
+rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, 
_]].shuffleHandle
+  rdd1.map {
+case (x, _) if (x == 1) =>
+  throw new FetchFailedException(
+BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, 
"test")
+case (x, _) => x
+  }.count()
+}
+
+def successJob: Unit = {
+  object FailThisAttempt {
+val _fail = new AtomicBoolean(true)
+  }
+  val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 
1)).groupByKey()
+  val shuffleHandle =
+rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, 
_]].shuffleHandle
+  rdd1.map {
+case (x, _) if (x == 1) && FailThisAttempt._fail.getAndSet(false) 
=>
+  throw new FetchFailedException(
+BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, 
"test")
+  }
+}
+
+failAfter(60.seconds) {
+  val e = intercept[SparkException] {
+fetchFailJob
+  }
+  
assert(e.getMessage.contains("org.apache.spark.shuffle.FetchFailedException"))
+}
+
+// The following job that fails due to fetching failure will hang 
without
+// the fix for SPARK-17644
+failAfter(60.seconds) {
--- End diff --

I think a shorter timeout would be appropriate here to avoid slow-ness when 
this fails...maybe 10 seconds? That still seems plenty conservative since the 
resubmit timeout is 200 millis.


---
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 #15213: [SPARK-17644] [CORE] Do not add failedStages when...

2016-09-27 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/15213#discussion_r80845459
  
--- Diff: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
@@ -2105,6 +2107,59 @@ class DAGSchedulerSuite extends SparkFunSuite with 
LocalSparkContext with Timeou
 assert(scheduler.getShuffleDependencies(rddE) === Set(shuffleDepA, 
shuffleDepC))
   }
 
+  test("After one stage is aborted for too many failed attempts, 
subsequent stages" +
+"still behave correctly on fetch failures") {
+def fetchFailJob: Unit = {
--- End diff --

to make this a little more descriptive / easy to read, how about calling 
the helper "runJobWithPersistentFetchFailure" and then add a comment that says 
"Runs a job that always encounters a fetch failure, so should eventually be 
aborted."


---
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 #15213: [SPARK-17644] [CORE] Do not add failedStages when...

2016-09-27 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/15213#discussion_r80845506
  
--- Diff: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
@@ -2105,6 +2107,59 @@ class DAGSchedulerSuite extends SparkFunSuite with 
LocalSparkContext with Timeou
 assert(scheduler.getShuffleDependencies(rddE) === Set(shuffleDepA, 
shuffleDepC))
   }
 
+  test("After one stage is aborted for too many failed attempts, 
subsequent stages" +
+"still behave correctly on fetch failures") {
+def fetchFailJob: Unit = {
+  val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 
1)).groupByKey()
+  val shuffleHandle =
+rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, 
_]].shuffleHandle
+  rdd1.map {
+case (x, _) if (x == 1) =>
+  throw new FetchFailedException(
+BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, 
"test")
+case (x, _) => x
+  }.count()
+}
+
+def successJob: Unit = {
--- End diff --

and for this perhaps call it "runJobWithTemporaryFetchFailure" and then 
comment saying "Runs a job that encounters a single fetch failure but succeeds 
on the second attempt"


---
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 #15213: [SPARK-17644] [CORE] Do not add failedStages when...

2016-09-27 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/15213#discussion_r80845188
  
--- Diff: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
@@ -2105,6 +2107,59 @@ class DAGSchedulerSuite extends SparkFunSuite with 
LocalSparkContext with Timeou
 assert(scheduler.getShuffleDependencies(rddE) === Set(shuffleDepA, 
shuffleDepC))
   }
 
+  test("After one stage is aborted for too many failed attempts, 
subsequent stages" +
+"still behave correctly on fetch failures") {
+def fetchFailJob: Unit = {
+  val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 
1)).groupByKey()
+  val shuffleHandle =
+rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, 
_]].shuffleHandle
+  rdd1.map {
+case (x, _) if (x == 1) =>
+  throw new FetchFailedException(
+BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, 
"test")
+case (x, _) => x
+  }.count()
+}
+
+def successJob: Unit = {
+  object FailThisAttempt {
+val _fail = new AtomicBoolean(true)
+  }
+  val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 
1)).groupByKey()
+  val shuffleHandle =
+rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, 
_]].shuffleHandle
+  rdd1.map {
+case (x, _) if (x == 1) && FailThisAttempt._fail.getAndSet(false) 
=>
+  throw new FetchFailedException(
+BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, 
"test")
+  }
+}
+
+failAfter(60.seconds) {
+  val e = intercept[SparkException] {
+fetchFailJob
+  }
+  
assert(e.getMessage.contains("org.apache.spark.shuffle.FetchFailedException"))
+}
+
+// The following job that fails due to fetching failure will hang 
without
+// the fix for SPARK-17644
+failAfter(60.seconds) {
+  val e = intercept[SparkException] {
+fetchFailJob
+  }
+  
assert(e.getMessage.contains("org.apache.spark.shuffle.FetchFailedException"))
+}
+
+failAfter(60.seconds) {
+  try {
+successJob
+  } catch {
+case e: Throwable => fail("this job should success")
--- End diff --

Can you make this a little more descriptive -- maybe "A job with one fetch 
failure should eventually succeed"?


---
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 #15213: [SPARK-17644] [CORE] Do not add failedStages when...

2016-09-27 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/15213#discussion_r80844742
  
--- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
---
@@ -1256,11 +1257,13 @@ class DAGScheduler(
   if (disallowStageRetryForTest) {
 abortStage(failedStage, "Fetch failure will not retry stage 
due to testing config",
   None)
+abortedStage = true
   } else if 
(failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)) {
 abortStage(failedStage, s"$failedStage (${failedStage.name}) " 
+
   s"has failed the maximum allowable number of " +
   s"times: ${Stage.MAX_CONSECUTIVE_FETCH_FAILURES}. " +
   s"Most recent failure reason: ${failureMessage}", None)
+abortedStage = true
   } else if (failedStages.isEmpty) {
--- End diff --

Instead of having the abortedStage variable, how about re-writing the "else 
if" statement to be:

else {
  if (failedStages.isEmpty) {
... stuff currently in else-if ...
  }
  failedStages += failedStage
  failedStages += mapStage
}

That eliminates the confusion of multiple abortStage variables, as @zsxwing 
pointed out, and also makes the relationship between (i) adding the stage to 
failed stages and (ii) scheduling the Resubmit event more clear.


---
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 #15213: [SPARK-17644] [CORE] Do not add failedStages when...

2016-09-27 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/15213#discussion_r80844841
  
--- Diff: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
@@ -2105,6 +2107,59 @@ class DAGSchedulerSuite extends SparkFunSuite with 
LocalSparkContext with Timeou
 assert(scheduler.getShuffleDependencies(rddE) === Set(shuffleDepA, 
shuffleDepC))
   }
 
+  test("After one stage is aborted for too many failed attempts, 
subsequent stages" +
+"still behave correctly on fetch failures") {
--- End diff --

Can you add the JIRA number here? That helps with tracking tests in the 
future.  So something like "[SPARK-17644] After one stage is aborted..."


---
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 #15213: [SPARK-17644] [CORE] Do not add failedStages when...

2016-09-27 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/15213#discussion_r80844780
  
--- Diff: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
@@ -31,6 +32,7 @@ import org.apache.spark._
 import org.apache.spark.broadcast.BroadcastManager
 import org.apache.spark.rdd.RDD
 import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
+import org.apache.spark.shuffle.FetchFailedException
--- End diff --

nit: can you group this with the next import (so import 
org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException}


---
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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15263
  
@gatorsmile Would this be sensible if I submit another PR to clean up 
options for https://github.com/apache/spark/pull/15263#issuecomment-250072008 
later as strictly the purpose of this PR is to clean up `JdbcRelationProvider`? 

I'd like to unify all the options within `JDBCOptions` and maybe I could 
fix the documentation together there if you think it makes sense.


---
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 issue #15257: [SPARK-17683][SQL] Support ArrayType in Literal.apply

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15257: [SPARK-17683][SQL] Support ArrayType in Literal.apply

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15257
  
**[Test build #66022 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66022/consoleFull)**
 for PR 15257 at commit 
[`c57d93e`](https://github.com/apache/spark/commit/c57d93e879cbfe6ad673f1c9cd31277c659f766d).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15257: [SPARK-17683][SQL] Support ArrayType in Literal.apply

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelatio...

2016-09-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15263#discussion_r80845381
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
 ---
@@ -50,67 +51,48 @@ class JdbcRelationProvider extends 
CreatableRelationProvider
 JDBCRelation(jdbcOptions.url, jdbcOptions.table, parts, 
properties)(sqlContext.sparkSession)
   }
 
-  /*
-   * The following structure applies to this code:
-   * |tableExists|  !tableExists
-   
*
-   * Ignore  | BaseRelation  | CreateTable, saveTable, 
BaseRelation
-   * ErrorIfExists   | ERROR | CreateTable, saveTable, 
BaseRelation
-   * Overwrite*  | (DropTable, CreateTable,) | CreateTable, saveTable, 
BaseRelation
-   * | saveTable, BaseRelation   |
-   * Append  | saveTable, BaseRelation   | CreateTable, saveTable, 
BaseRelation
-   *
-   * *Overwrite & tableExists with truncate, will not drop & create, but 
instead truncate
-   */
   override def createRelation(
   sqlContext: SQLContext,
   mode: SaveMode,
   parameters: Map[String, String],
-  data: DataFrame): BaseRelation = {
-val jdbcOptions = new JDBCOptions(parameters)
-val url = jdbcOptions.url
-val table = jdbcOptions.table
-
+  df: DataFrame): BaseRelation = {
+val options = new JDBCOptions(parameters)
+val url = options.url
+val table = options.table
+val createTableOptions = options.createTableOptions
+val isTruncate = options.isTruncate
 val props = new Properties()
 props.putAll(parameters.asJava)
-val conn = JdbcUtils.createConnectionFactory(url, props)()
 
+val conn = JdbcUtils.createConnectionFactory(url, props)()
 try {
   val tableExists = JdbcUtils.tableExists(conn, url, table)
+  if (tableExists) {
+mode match {
+  case SaveMode.Overwrite =>
+if (isTruncate && 
isCascadingTruncateTable(url).contains(false)) {
+  // In this case, we should truncate table and then load.
+  truncateTable(conn, table)
+  saveTable(df, url, table, props)
+} else {
+  // Otherwise, do not truncate but just drop.
--- End diff --

This comment looks weird. In RDBMS, users might do `drop and recreate` a 
table, instead of truncating it. Thus, please change it to

`// Otherwise, do not truncate but just drop and recreate`




---
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 issue #15257: [SPARK-17683][SQL] Support ArrayType in Literal.apply

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15257
  
**[Test build #66022 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66022/consoleFull)**
 for PR 15257 at commit 
[`c57d93e`](https://github.com/apache/spark/commit/c57d93e879cbfe6ad673f1c9cd31277c659f766d).


---
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 #15257: [SPARK-17683][SQL] Support ArrayType in Literal.a...

2016-09-27 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/15257#discussion_r80845375
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
 ---
@@ -52,13 +53,51 @@ object Literal {
 case t: Timestamp => Literal(DateTimeUtils.fromJavaTimestamp(t), 
TimestampType)
 case d: Date => Literal(DateTimeUtils.fromJavaDate(d), DateType)
 case a: Array[Byte] => Literal(a, BinaryType)
+case a: Array[_] =>
+  val elementType = 
componentTypeToDataType(a.getClass.getComponentType())
+  val dataType = ArrayType(elementType)
+  val convert = 
CatalystTypeConverters.createToCatalystConverter(dataType)
+  Literal(convert(a), dataType)
 case i: CalendarInterval => Literal(i, CalendarIntervalType)
 case null => Literal(null, NullType)
 case v: Literal => v
 case _ =>
   throw new RuntimeException("Unsupported literal type " + v.getClass 
+ " " + v)
   }
 
+  private def componentTypeToDataType(clz: Class[_]): DataType = clz match 
{
--- End diff --

@jaceklaskowski I checked these similar code though, I couldn't find better 
ideas to use reflection stuffs here in `ScalaReflection` because 
`Literal#apply` cannot catch type signatures in runtime. If possible, any 
better idea?


---
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 issue #14897: [SPARK-17338][SQL] add global temp view

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14897
  
**[Test build #66021 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66021/consoleFull)**
 for PR 14897 at commit 
[`a683fba`](https://github.com/apache/spark/commit/a683fba7fdff0dd101be231a84a88fb5b8f3aef4).


---
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 issue #15254: [SPARK-17679] [PYSPARK] remove unnecessary Py4J ListConv...

2016-09-27 Thread JasonMWhite
Github user JasonMWhite commented on the issue:

https://github.com/apache/spark/pull/15254
  
@JoshRosen @lins05 As requested, I've removed all remaining explicit 
mentions of `ListConverter` and `MapConverter` as they seemed to all be doing 
the same thing - getting around https://github.com/bartdag/py4j/issues/161 and 
https://github.com/bartdag/py4j/issues/160.

I'm not familiar with the code in pyspark-ml and pyspark-mllib, but it 
seemed straightforward and didn't introduce any regressions in the tests.


---
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 #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelatio...

2016-09-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15263#discussion_r80844741
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
 ---
@@ -50,67 +51,48 @@ class JdbcRelationProvider extends 
CreatableRelationProvider
 JDBCRelation(jdbcOptions.url, jdbcOptions.table, parts, 
properties)(sqlContext.sparkSession)
   }
 
-  /*
-   * The following structure applies to this code:
-   * |tableExists|  !tableExists
-   
*
-   * Ignore  | BaseRelation  | CreateTable, saveTable, 
BaseRelation
-   * ErrorIfExists   | ERROR | CreateTable, saveTable, 
BaseRelation
-   * Overwrite*  | (DropTable, CreateTable,) | CreateTable, saveTable, 
BaseRelation
-   * | saveTable, BaseRelation   |
-   * Append  | saveTable, BaseRelation   | CreateTable, saveTable, 
BaseRelation
-   *
-   * *Overwrite & tableExists with truncate, will not drop & create, but 
instead truncate
-   */
   override def createRelation(
   sqlContext: SQLContext,
   mode: SaveMode,
   parameters: Map[String, String],
-  data: DataFrame): BaseRelation = {
-val jdbcOptions = new JDBCOptions(parameters)
-val url = jdbcOptions.url
-val table = jdbcOptions.table
-
+  df: DataFrame): BaseRelation = {
+val options = new JDBCOptions(parameters)
+val url = options.url
+val table = options.table
+val createTableOptions = options.createTableOptions
+val isTruncate = options.isTruncate
 val props = new Properties()
 props.putAll(parameters.asJava)
-val conn = JdbcUtils.createConnectionFactory(url, props)()
 
+val conn = JdbcUtils.createConnectionFactory(url, props)()
 try {
   val tableExists = JdbcUtils.tableExists(conn, url, table)
+  if (tableExists) {
+mode match {
+  case SaveMode.Overwrite =>
+if (isTruncate && 
isCascadingTruncateTable(url).contains(false)) {
+  // In this case, we should truncate table and then load.
+  truncateTable(conn, table)
+  saveTable(df, url, table, props)
+} else {
+  // Otherwise, do not truncate but just drop.
+  dropTable(conn, table)
+  createTable(df, url, table, createTableOptions, conn)
+  saveTable(df, url, table, props)
+}
+  case SaveMode.Append =>
+saveTable(df, url, table, props)
 
-  val (doCreate, doSave) = (mode, tableExists) match {
-case (SaveMode.Ignore, true) => (false, false)
-case (SaveMode.ErrorIfExists, true) => throw new AnalysisException(
-  s"Table or view '$table' already exists, and SaveMode is set to 
ErrorIfExists.")
-case (SaveMode.Overwrite, true) =>
-  if (jdbcOptions.isTruncate && 
JdbcUtils.isCascadingTruncateTable(url) == Some(false)) {
-JdbcUtils.truncateTable(conn, table)
-(false, true)
-  } else {
-JdbcUtils.dropTable(conn, table)
-(true, true)
-  }
-case (SaveMode.Append, true) => (false, true)
-case (_, true) => throw new IllegalArgumentException(s"Unexpected 
SaveMode, '$mode'," +
-  " for handling existing tables.")
-case (_, false) => (true, true)
-  }
+  case SaveMode.ErrorIfExists =>
+throw new AnalysisException(
+  s"Table or view '$table' already exists, and SaveMode is set 
to ErrorIfExists.")
 
-  if (doCreate) {
-val schema = JdbcUtils.schemaString(data, url)
-// To allow certain options to append when create a new table, 
which can be
-// table_options or partition_options.
-// E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT 
CHARSET=utf8"
-val createtblOptions = jdbcOptions.createTableOptions
-val sql = s"CREATE TABLE $table ($schema) $createtblOptions"
-val statement = conn.createStatement
-try {
-  statement.executeUpdate(sql)
-} finally {
-  statement.close()
+  case SaveMode.Ignore => // Just ignore this case.
--- End diff --

Please explicitly explain the behavior. `// Just ignore this case` is 
almost useless.


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

[GitHub] spark issue #15275: [SPARK-17702][SQL] Code generation including too many mu...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15275: [SPARK-17702][SQL] Code generation including too many mu...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15275
  
**[Test build #66020 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66020/consoleFull)**
 for PR 15275 at commit 
[`858a3ec`](https://github.com/apache/spark/commit/858a3ec54f78b4865c19ec46aebc9320b36f02b5).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15275: [SPARK-17702][SQL] Code generation including too many mu...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15273
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15273
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15263
  
Good idea. Yes I will.


---
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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/15263
  
Regarding the JDBC options, which properties are used for the table 
reading? Which ones are used for writing? Can you please improve the document 
and make users easily find the answer?


---
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 issue #15258: [SPARK-17689][SQL][STREAMING][WIP] added excludeFiles op...

2016-09-27 Thread ScrapCodes
Github user ScrapCodes commented on the issue:

https://github.com/apache/spark/pull/15258
  
@frreiss That is correct, I will look into it. Moving the PR to WIP, as at 
least I would like to document these situations.


---
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 issue #15239: [SPARK-17665][SPARKR] Support options/mode all for read/...

2016-09-27 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/15239
  
re: test - I mean for each function we are adding the `...` param, that we 
have a test for calling it without the extra stuff, ie.
one test with
```
 write.json(df, jsonPath)
```

and one test with
```
 write.json(df, jsonPath, compression = 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 issue #15275: [SPARK-17702][SQL] Code generation including too many mu...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15275
  
**[Test build #66020 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66020/consoleFull)**
 for PR 15275 at commit 
[`858a3ec`](https://github.com/apache/spark/commit/858a3ec54f78b4865c19ec46aebc9320b36f02b5).


---
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 #15275: [SPARK-17702][SQL] Code generation including too ...

2016-09-27 Thread ueshin
GitHub user ueshin opened a pull request:

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

[SPARK-17702][SQL] Code generation including too many mutable states 
exceeds JVM size limit.

## What changes were proposed in this pull request?

Code generation including too many mutable states exceeds JVM size limit to 
extract values from `references` into fields in the constructor.
We should split the generated extractions in the constructor into smaller 
functions.

## How was this patch tested?

I added some tests to check if the generated codes for the expressions 
exceed or not.


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

$ git pull https://github.com/ueshin/apache-spark issues/SPARK-17702

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

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


commit 4c78ca91d0c0e096e5b0668f35a0d24457e36586
Author: Takuya UESHIN 
Date:   2016-09-28T04:19:34Z

Add a test to reproduce the issue.

commit 858a3ec54f78b4865c19ec46aebc9320b36f02b5
Author: Takuya UESHIN 
Date:   2016-09-28T04:26:38Z

Split wide constructor into blocks due to JVM code size limit.




---
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 #15239: [SPARK-17665][SPARKR] Support options/mode all fo...

2016-09-27 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/15239#discussion_r80841590
  
--- Diff: R/pkg/R/utils.R ---
@@ -342,7 +342,8 @@ varargsToStrEnv <- function(...) {
   for (name in names(pairs)) {
 value <- pairs[[name]]
 if (!(is.logical(value) || is.numeric(value) || is.character(value) || 
is.null(value))) {
-  stop("value[", value, "] in key[", name, "] is not convertable to 
string.")
+  stop(paste0("Unsupported type for ", name, " : ", class(value),
+   ". Supported types are logical, numeric, character and null."))
--- End diff --

`NULL` instead of `null`


---
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 #15239: [SPARK-17665][SPARKR] Support options/mode all fo...

2016-09-27 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/15239#discussion_r80841566
  
--- Diff: R/pkg/R/SQLContext.R ---
@@ -328,6 +328,7 @@ setMethod("toDF", signature(x = "RDD"),
 #' It goes through the entire dataset once to determine the schema.
 #'
 #' @param path Path of file to read. A vector of multiple paths is allowed.
+#' @param ... additional external data source specific named properties.
--- End diff --

this is odd - was it not complaining about this missing?


---
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 issue #14646: [SPARK-17058] [build] Add maven snapshots-and-staging pr...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14646
  
**[Test build #66019 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66019/consoleFull)**
 for PR 14646 at commit 
[`09d96be`](https://github.com/apache/spark/commit/09d96bec29c9ac9a58966c603c2c6b3b642eb9d2).


---
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 #15239: [SPARK-17665][SPARKR] Support options/mode all fo...

2016-09-27 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/15239#discussion_r80841401
  
--- Diff: R/pkg/R/DataFrame.R ---
@@ -55,6 +55,19 @@ setMethod("initialize", "SparkDataFrame", 
function(.Object, sdf, isCached) {
   .Object
 })
 
+#' Set options/mode and then return the write object
+#' @noRd
+setWriteOptions <- function(write, path = NULL, mode = 'error', ...) {
+options <- varargsToStrEnv(...)
+if (!is.null(path)) {
+  options[["path"]] <- path
+}
+jmode <- convertToJSaveMode(mode)
+write <- callJMethod(write, "mode", jmode)
+write <- callJMethod(write, "options", options)
+write
--- End diff --

do you think if it make sense to have a generic write method? ie. include 
`write <- callJMethod(x@sdf, "write")` and `invisible(callJMethod(write, 
source, path))`


---
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 #15239: [SPARK-17665][SPARKR] Support options/mode all fo...

2016-09-27 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/15239#discussion_r80841143
  
--- Diff: R/pkg/R/utils.R ---
@@ -334,6 +334,27 @@ varargsToEnv <- function(...) {
   env
 }
 
+# Utility function to capture the varargs into environment object but all 
values are converted
+# into string.
+varargsToStrEnv <- function(...) {
+  pairs <- list(...)
+  env <- new.env()
+  for (name in names(pairs)) {
+value <- pairs[[name]]
+if (!(is.logical(value) || is.numeric(value) || is.character(value) || 
is.null(value))) {
+  stop("value[", value, "] in key[", name, "] is not convertable to 
string.")
--- End diff --

I think "... logical, character, numeric and NULL" as these are the R names.


---
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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15273
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 #15232: [SPARK-17499][SPARKR][FOLLOWUP] Check null first ...

2016-09-27 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #14818: [SPARK-17157][SPARKR][WIP]: Add multiclass logistic regr...

2016-09-27 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/14818
  
make sense, I think then we should have a separate function 


---
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 issue #15247: [SPARK-17672] Spark 2.0 history server web Ui takes too ...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15247: [SPARK-17672] Spark 2.0 history server web Ui takes too ...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15247
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15247: [SPARK-17672] Spark 2.0 history server web Ui takes too ...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15247
  
**[Test build #66013 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66013/consoleFull)**
 for PR 15247 at commit 
[`7e437a1`](https://github.com/apache/spark/commit/7e437a1abf9b2a9461400eecea375ae6cd6043b7).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15272: [SPARK-17698] [SQL] Join predicates should not contain f...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15272
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15272: [SPARK-17698] [SQL] Join predicates should not contain f...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15272: [SPARK-17698] [SQL] Join predicates should not contain f...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15258: [SPARK-17689][SQL][STREAMING] added excludeFiles option ...

2016-09-27 Thread frreiss
Github user frreiss commented on the issue:

https://github.com/apache/spark/pull/15258
  
This change allows FileInputStream to consume partial outputs of a system 
such as Hadoop or another copy of Spark, provided that the system adheres 
rigidly to the write policy of recent versions of Hadoop. That is: First, write 
to a temporary file. Then close and flush the temporary file. Then rename the 
temporary file, using one of the newer, atomic HDFS APIs for renaming files. I 
worry that users might write data in a subtly different way that does not 
follow this procedure 100%, which could result in Spark reading incorrect data 
every once in a while. I recommend documenting under exactly what conditions 
the "ignore temporary files" option guarantees correct behavior.

Also, it would be a good idea to include a mode in which FileInputStream 
will ignore a directory of files until the special file _SUCCESS appears, 
indicating that the directory is complete. Otherwise, Spark could end up 
consuming partial results from failed jobs.


---
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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15263
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15263
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15263: [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvid...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 #15258: [SPARK-17689][SQL][STREAMING] added excludeFiles ...

2016-09-27 Thread frreiss
Github user frreiss commented on a diff in the pull request:

https://github.com/apache/spark/pull/15258#discussion_r80838376
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 ---
@@ -50,6 +50,19 @@ class ListingFileCatalog(
 
   refresh()
 
+  /**
+   * Often HDFS create temporary files while copying to a new directory or 
writing new content.
+   * These files are unintentionally picked up by streaming - causing job 
failures. This option lets
+   * HDFS skip these files matching the configured regex-patterns from 
being picked up by Streaming
+   * Job.
+   */
+  private lazy val excludeFiles: Set[String] = parameters
+.getOrElse("excludeFiles", ".*._COPYING_,_temporary").split(",").toSet
+
+  private def isExcludedFile(path: Path): Boolean = {
+excludeFiles.map(path.getName.matches).fold(false)(_ || _)
--- End diff --

Probably better to precompile a regex here, as this method will be called a 
lot (100X per second X number of files under the root directory, by default)


---
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 issue #15271: [SPARK-17666] Ensure that RecordReaders are closed by da...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15271
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15271: [SPARK-17666] Ensure that RecordReaders are closed by da...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15271: [SPARK-17666] Ensure that RecordReaders are closed by da...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15271
  
**[Test build #66009 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66009/consoleFull)**
 for PR 15271 at commit 
[`c0621db`](https://github.com/apache/spark/commit/c0621dbbd558e3715ea3b0c250913c1b94e34478).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class HadoopFileLinesReader(`
  * `class RecordReaderIterator[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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15273
  
**[Test build #66018 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66018/consoleFull)**
 for PR 15273 at commit 
[`e2dfb25`](https://github.com/apache/spark/commit/e2dfb25aeef021788208a0f05420d211916d2db5).


---
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 issue #15213: [SPARK-17644] [CORE] Do not add failedStages when abortS...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15213
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15213: [SPARK-17644] [CORE] Do not add failedStages when abortS...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15213: [SPARK-17644] [CORE] Do not add failedStages when abortS...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 #15274: [SPARK-17699] Support for parsing JSON string col...

2016-09-27 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15274#discussion_r80837055
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -467,3 +469,26 @@ case class JsonTuple(children: Seq[Expression])
   }
 }
 
+/**
+ * Converts an json input string to a [[StructType]] with the specified 
schema.
+ */
+case class JsonToStruct(schema: StructType, options: Map[String, String], 
child: Expression)
--- End diff --

Ah, yes, it definitly should.  Let me update.


---
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 issue #15273: [SPARK-17673] [SQL] Incorrect exchange reuse with RowDat...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15273
  
**[Test build #66017 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66017/consoleFull)**
 for PR 15273 at commit 
[`f0df3a0`](https://github.com/apache/spark/commit/f0df3a005be301ac627beb53a233ece06269ced8).


---
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 issue #15267: [SPARK-17667] [YARN][WIP]Make locking fine grained in Ya...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15267
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15267: [SPARK-17667] [YARN][WIP]Make locking fine grained in Ya...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15267: [SPARK-17667] [YARN][WIP]Make locking fine grained in Ya...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15267
  
**[Test build #66015 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66015/consoleFull)**
 for PR 15267 at commit 
[`859718c`](https://github.com/apache/spark/commit/859718c95e3864855be98c5d09e74c652eda31b9).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 #15274: [SPARK-17699] Support for parsing JSON string col...

2016-09-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/15274#discussion_r80836637
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -467,3 +469,26 @@ case class JsonTuple(children: Seq[Expression])
   }
 }
 
+/**
+ * Converts an json input string to a [[StructType]] with the specified 
schema.
+ */
+case class JsonToStruct(schema: StructType, options: Map[String, String], 
child: Expression)
--- End diff --

Should this override `ExpectsInputTypes`?


---
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 issue #14638: [SPARK-11374][SQL] Support `skip.header.line.count` opti...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14638
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #14638: [SPARK-11374][SQL] Support `skip.header.line.count` opti...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #14638: [SPARK-11374][SQL] Support `skip.header.line.count` opti...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14638
  
**[Test build #66008 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66008/consoleFull)**
 for PR 14638 at commit 
[`51a83cf`](https://github.com/apache/spark/commit/51a83cfaccf967ea5329c5609dc606c1527b364b).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #14897: [SPARK-17338][SQL] add global temp view

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #14897: [SPARK-17338][SQL] add global temp view

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #14897: [SPARK-17338][SQL] add global temp view

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14897
  
**[Test build #66007 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66007/consoleFull)**
 for PR 14897 at commit 
[`6183400`](https://github.com/apache/spark/commit/6183400275837f51a84b6259e7aa5e011d77c2ea).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15102: [SPARK-17346][SQL] Add Kafka source for Structured Strea...

2016-09-27 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/15102
  
FYI: #15274 adds support for parsing JSON from the key/value into a Spark 
SQL `StructType`


---
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 issue #15274: [SPARK-17699] Support for parsing JSON string columns

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15274: [SPARK-17699] Support for parsing JSON string columns

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15274
  
**[Test build #66016 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66016/consoleFull)**
 for PR 15274 at commit 
[`62f56a7`](https://github.com/apache/spark/commit/62f56a7e4529b35f58a229097b012bc984fd458f).
 * This patch **fails Python style tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15274: [SPARK-17699] Support for parsing JSON string columns

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15274: [SPARK-17699] Support for parsing JSON string columns

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15274
  
**[Test build #66016 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66016/consoleFull)**
 for PR 15274 at commit 
[`62f56a7`](https://github.com/apache/spark/commit/62f56a7e4529b35f58a229097b012bc984fd458f).


---
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 #15274: [SPARK-17699] Support for parsing JSON string col...

2016-09-27 Thread marmbrus
GitHub user marmbrus opened a pull request:

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

[SPARK-17699] Support for parsing JSON string columns

Spark SQL has great support for reading text files that contain JSON data.  
However, in many cases the JSON data is just one column amongst others.  This 
is particularly true when reading from sources such as Kafka.  This PR adds a 
new functions `from_json` that converts a string column into a nested 
`StructType` with a user specified schema.

Example usage:
```scala
val df = Seq("""{"a": 1}""").toDS()
val schema = new StructType().add("a", IntegerType)

df.select(from_json($"value", schema) as 'json) // => [json: ]
```

This PR adds support for java, scala and python.  I leveraged our existing 
JSON parsing support by moving it into catalyst (so that we could define 
expressions using it).  I left SQL out for now, because I'm not sure how users 
would specify a schema.

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

$ git pull https://github.com/marmbrus/spark jsonParser

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

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


commit 62f56a7e4529b35f58a229097b012bc984fd458f
Author: Michael Armbrust 
Date:   2016-09-28T02:49:22Z

[SPARK-17699] Support for parsing JSON string columns




---
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 issue #15267: [SPARK-17667] [YARN][WIP]Make locking fine grained in Ya...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15267
  
**[Test build #66015 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66015/consoleFull)**
 for PR 15267 at commit 
[`859718c`](https://github.com/apache/spark/commit/859718c95e3864855be98c5d09e74c652eda31b9).


---
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 issue #11119: [SPARK-10780][ML] Add an initial model to kmeans

2016-09-27 Thread sethah
Github user sethah commented on the issue:

https://github.com/apache/spark/pull/9
  
@yinxusen Looking good. I left a few small comments, and we should take 
care of the checking of initial model params in the read/write test now. After 
that, I think it will be ready to merge. 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 #11119: [SPARK-10780][ML] Add an initial model to kmeans

2016-09-27 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/9#discussion_r80834369
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala ---
@@ -300,15 +301,23 @@ private[ml] object DefaultParamsWriter {
   paramMap: Option[JValue] = None): String = {
 val uid = instance.uid
 val cls = instance.getClass.getName
-val params = 
instance.extractParamMap().toSeq.asInstanceOf[Seq[ParamPair[Any]]]
+val params = instance.extractParamMap().toSeq
+  .filter(_.param.name != 
"initialModel").asInstanceOf[Seq[ParamPair[Any]]]
 val jsonParams = paramMap.getOrElse(render(params.map { case 
ParamPair(p, v) =>
   p.name -> parse(p.jsonEncode(v))
 }.toList))
+// If the instance has an "initialModel" param and the param is 
defined, then the initial model
+// will be saved along with the instance.
+val initialModelFlag =
+  instance.hasParam("initialModel") && 
instance.isDefined(instance.getParam("initialModel"))
 val basicMetadata = ("class" -> cls) ~
   ("timestamp" -> System.currentTimeMillis()) ~
   ("sparkVersion" -> sc.version) ~
   ("uid" -> uid) ~
-  ("paramMap" -> jsonParams)
+  ("paramMap" -> jsonParams) ~
+  // TODO: Figure out more robust way to detect the existing of the 
initialModel.
--- End diff --

This test should probably be called "Initialize using a trained model"


---
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 #11119: [SPARK-10780][ML] Add an initial model to kmeans

2016-09-27 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/9#discussion_r80835092
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala ---
@@ -17,12 +17,19 @@
 
 package org.apache.spark.ml.clustering
 
+import scala.util.Random
+
 import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.Model
--- End diff --

unused.


---
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 #11119: [SPARK-10780][ML] Add an initial model to kmeans

2016-09-27 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/9#discussion_r80835086
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala ---
@@ -17,12 +17,19 @@
 
 package org.apache.spark.ml.clustering
 
+import scala.util.Random
+
 import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.Model
 import org.apache.spark.ml.linalg.{Vector, Vectors}
-import org.apache.spark.ml.util.DefaultReadWriteTest
-import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans}
+import org.apache.spark.ml.param.{ParamMap, ParamPair}
+import org.apache.spark.ml.util.{DefaultReadWriteTest, Identifiable}
+import org.apache.spark.ml.util.TestingUtils._
+import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans, 
KMeansModel => MLlibKMeansModel}
+import org.apache.spark.mllib.linalg.{Vectors => MLlibVectors}
 import org.apache.spark.mllib.util.MLlibTestSparkContext
 import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.types.StructType
--- End diff --

unused.


---
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 #11119: [SPARK-10780][ML] Add an initial model to kmeans

2016-09-27 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/9#discussion_r80835233
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala ---
@@ -17,12 +17,19 @@
 
 package org.apache.spark.ml.clustering
 
+import scala.util.Random
+
 import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.Model
 import org.apache.spark.ml.linalg.{Vector, Vectors}
-import org.apache.spark.ml.util.DefaultReadWriteTest
-import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans}
+import org.apache.spark.ml.param.{ParamMap, ParamPair}
+import org.apache.spark.ml.util.{DefaultReadWriteTest, Identifiable}
--- End diff --

unused.


---
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 issue #15267: [SPARK-17667] [YARN][WIP]Make locking fine grained in Ya...

2016-09-27 Thread vanzin
Github user vanzin commented on the issue:

https://github.com/apache/spark/pull/15267
  
ok to test


---
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 #15189: [SPARK-17549][sql] Coalesce cached relation stats...

2016-09-27 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15189#discussion_r80835147
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
 ---
@@ -232,4 +232,29 @@ class InMemoryColumnarQuerySuite extends QueryTest 
with SharedSQLContext {
 val columnTypes2 = List.fill(length2)(IntegerType)
 val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2)
   }
+
+  test("SPARK-17549: cached table size should be correctly calculated") {
+val data = spark.sparkContext.parallelize(1 to 10, 5).map { i => (i, 
i.toLong) }
+  .toDF("col1", "col2")
+val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan
+val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None)
+
+// Materialize the data.
+val expectedAnswer = data.collect()
+checkAnswer(cached, expectedAnswer)
+
+// Check that the right size was calculated.
+val expectedColSizes = expectedAnswer.size * (INT.defaultSize + 
LONG.defaultSize)
+assert(cached.statistics.sizeInBytes === expectedColSizes)
+
+// Create a projection of the cached data and make sure the statistics 
are correct.
+val projected = cached.withOutput(Seq(plan.output.last))
+assert(projected.statistics.sizeInBytes === expectedAnswer.size * 
LONG.defaultSize)
--- End diff --

Thanks for confirming. So we should be fine with the previous 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 issue #15248: [SPARK-17671] Spark 2.0 history server summary page is s...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15248: [SPARK-17671] Spark 2.0 history server summary page is s...

2016-09-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15248
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #15248: [SPARK-17671] Spark 2.0 history server summary page is s...

2016-09-27 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15248
  
**[Test build #66003 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66003/consoleFull)**
 for PR 15248 at commit 
[`6c84507`](https://github.com/apache/spark/commit/6c84507fc60fb021af3593a7b87ad6e9fb10e752).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 #15189: [SPARK-17549][sql] Coalesce cached relation stats...

2016-09-27 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15189#discussion_r80834862
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
 ---
@@ -232,4 +232,29 @@ class InMemoryColumnarQuerySuite extends QueryTest 
with SharedSQLContext {
 val columnTypes2 = List.fill(length2)(IntegerType)
 val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2)
   }
+
+  test("SPARK-17549: cached table size should be correctly calculated") {
+val data = spark.sparkContext.parallelize(1 to 10, 5).map { i => (i, 
i.toLong) }
+  .toDF("col1", "col2")
+val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan
+val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None)
+
+// Materialize the data.
+val expectedAnswer = data.collect()
+checkAnswer(cached, expectedAnswer)
+
+// Check that the right size was calculated.
+val expectedColSizes = expectedAnswer.size * (INT.defaultSize + 
LONG.defaultSize)
+assert(cached.statistics.sizeInBytes === expectedColSizes)
+
+// Create a projection of the cached data and make sure the statistics 
are correct.
+val projected = cached.withOutput(Seq(plan.output.last))
+assert(projected.statistics.sizeInBytes === expectedAnswer.size * 
LONG.defaultSize)
--- End diff --

I double checked the code. The `output` of an `InMemoryRelation` always 
represent the materialized dataset. So, it should not be a set of the 
underlying dataset's column set. When we scan this relation in 
`InMemoryTableScanExec`, we will push the selection to the scan. 

So, even we use `withOutput` in `CacheManager`'s `useCachedData`, we should 
be fine to still use the original stats because we are not changing the 
dataset. If you look at the implementation of this method
```
def useCachedData(plan: LogicalPlan): LogicalPlan = {
plan transformDown {
  case currentFragment =>
lookupCachedData(currentFragment)
  .map(_.cachedRepresentation.withOutput(currentFragment.output))
  .getOrElse(currentFragment)
}
  }
```
`lookupCachedData` is implemented using `sameResult`. So, we are just 
applying a equivalent output (attributes in this output list may have cosmetic 
variations but they should be equivalent to the original attributes of this 
dataset).

Although we may have different outputs, they are still representing the 
same dataset. So, seems it is fine if they have the same accumulator. 


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