[GitHub] spark pull request #17079: [SPARK-19748][SQL]refresh function has an wrong o...

2017-02-26 Thread windpiger
GitHub user windpiger opened a pull request:

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

[SPARK-19748][SQL]refresh function has an wrong order to do cache 
invalidate and regenerate the inmemory var for InMemoryFileIndex with 
FileStatusCache

## What changes were proposed in this pull request?

If we refresh a InMemoryFileIndex with a FileStatusCache, it will first use 
the FileStatusCache to re-generate the cachedLeafFiles etc, then call 
FileStatusCache.invalidateAll. 

While the order to do these two actions is wrong, this lead to the refresh 
action does not take effect.

```
  override def refresh(): Unit = {
refresh0()
fileStatusCache.invalidateAll()
  }

  private def refresh0(): Unit = {
val files = listLeafFiles(rootPaths)
cachedLeafFiles =
  new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => 
f.getPath -> f)
cachedLeafDirToChildrenFiles = 
files.toArray.groupBy(_.getPath.getParent)
cachedPartitionSpec = null
  }
```
## How was this patch tested?
unit test added

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

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

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

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


commit 552955a5293bc22da1cd644ffde90b883fc560e8
Author: windpiger 
Date:   2017-02-27T07:46:18Z

[SPARK-19748][SQL]refresh function has an wrong order to do cache 
invalidate and regenerate the inmemory var for InMemoryFileIndex with 
FileStatusCache

commit fd3bb21597809409e7f33796589c9178744063c5
Author: windpiger 
Date:   2017-02-27T07:53:46Z

modify the test case




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 #15505: [SPARK-18890][CORE] Move task serialization from ...

2017-02-26 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/15505#discussion_r103147221
  
--- Diff: 
core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala ---
@@ -23,7 +23,10 @@ import java.util.Properties
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.{HashMap, Map}
+import scala.util.control.NonFatal
 
+import org.apache.spark.internal.Logging
+import org.apache.spark.TaskNotSerializableException
--- End diff --

Can you do this?  Looks like this comment got forgotten.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16910: [SPARK-19575][SQL]Reading from or writing to a hi...

2017-02-26 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16910#discussion_r103143687
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala ---
@@ -83,6 +83,8 @@ class HadoopTableReader(
   private val _broadcastedHadoopConf =
 sparkSession.sparkContext.broadcast(new 
SerializableConfiguration(hadoopConf))
 
+  private lazy val STORAGE_HANDLER_KEY = "storage_handler"
--- End diff --

This can be removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16910: [SPARK-19575][SQL]Reading from or writing to a hi...

2017-02-26 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16910#discussion_r103143673
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala ---
@@ -114,22 +116,34 @@ class HadoopTableReader(
 val tablePath = hiveTable.getPath
 val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt)
 
-// logDebug("Table input: %s".format(tablePath))
-val ifc = hiveTable.getInputFormatClass
-  .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]]
-val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc)
+val locationPath = new Path(inputPathStr)
+val fs = locationPath.getFileSystem(broadcastedHadoopConf.value.value)
 
-val attrsWithIndex = attributes.zipWithIndex
-val mutableRow = new SpecificInternalRow(attributes.map(_.dataType))
+// if the location of the table which is not created by 'stored by' 
does not exist,
+// return an empty RDD
+// TODO: after SparkSQL supports 'stored by', we should check if this 
condition
+// is still proper.
+val storageHandler = 
hiveTable.getParameters.getOrDefault(STORAGE_HANDLER_KEY, null)
--- End diff --

Replace `STORAGE_HANDLER_KEY ` by `META_TABLE_STORAGE`, which has been 
defined in `hive_metastoreConstants`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17032: [SPARK-19460][SparkR]:Update dataset used in R documenta...

2017-02-26 Thread wangmiao1981
Github user wangmiao1981 commented on the issue:

https://github.com/apache/spark/pull/17032
  
@felixcheung I have made the changes per our review discussion. 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 issue #17001: [SPARK-19667][SQL]create table with hiveenabled in defau...

2017-02-26 Thread windpiger
Github user windpiger commented on the issue:

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


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

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



[GitHub] spark pull request #16677: [SPARK-19355][SQL] Use map output statistices to ...

2017-02-26 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/16677#discussion_r103140028
  
--- Diff: core/src/main/scala/org/apache/spark/MapOutputStatistics.scala ---
@@ -23,5 +23,9 @@ package org.apache.spark
  * @param shuffleId ID of the shuffle
  * @param bytesByPartitionId approximate number of output bytes for each 
map output partition
  *   (may be inexact due to use of compressed map statuses)
+ * @param numberOfOutput number of output for each pre-map output partition
  */
-private[spark] class MapOutputStatistics(val shuffleId: Int, val 
bytesByPartitionId: Array[Long])
+private[spark] class MapOutputStatistics(
+val shuffleId: Int,
+val bytesByPartitionId: Array[Long],
+val numberOfOutput: Array[Int])
--- End diff --

Ok. Use `Long` now


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

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



[GitHub] spark pull request #16677: [SPARK-19355][SQL] Use map output statistices to ...

2017-02-26 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/16677#discussion_r103140017
  
--- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
@@ -39,16 +40,18 @@ private[spark] sealed trait MapStatus {
* necessary for correctness, since block fetchers are allowed to skip 
zero-size blocks.
*/
   def getSizeForBlock(reduceId: Int): Long
+
+  def numberOfOutput: Int
--- End diff --

Use `Long` now.


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

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



[GitHub] spark pull request #17077: [SPARK-16931][PYTHON][SQL] Add Python wrapper for...

2017-02-26 Thread zero323
Github user zero323 commented on a diff in the pull request:

https://github.com/apache/spark/pull/17077#discussion_r103139360
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -2038,6 +2038,53 @@ def test_BinaryType_serialization(self):
 df = self.spark.createDataFrame(data, schema=schema)
 df.collect()
 
+def test_bucketed_write(self):
+data = [
+(1, "foo", 3.0), (2, "foo", 5.0),
+(3, "bar", -1.0), (4, "bar", 6.0),
+]
+df = self.spark.createDataFrame(data, ["x", "y", "z"])
+
+# Test write with one bucketing column
+df.write.bucketBy(3, 
"x").mode("overwrite").saveAsTable("pyspark_bucket")
+self.assertEqual(
+len([c for c in 
self.spark.catalog.listColumns("pyspark_bucket") if c.name == "x" and 
c.isBucket]),
--- End diff --

Indeed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17076: [SPARK-19745][ML] SVCAggregator captures coeffici...

2017-02-26 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17076#discussion_r103139054
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala ---
@@ -440,19 +440,9 @@ private class LinearSVCAggregator(
 
   private val numFeatures: Int = bcFeaturesStd.value.length
   private val numFeaturesPlusIntercept: Int = if (fitIntercept) 
numFeatures + 1 else numFeatures
-  private val coefficients: Vector = bcCoefficients.value
   private var weightSum: Double = 0.0
   private var lossSum: Double = 0.0
-  require(numFeaturesPlusIntercept == coefficients.size, s"Dimension 
mismatch. Coefficients " +
-s"length ${coefficients.size}, FeaturesStd length ${numFeatures}, 
fitIntercept: $fitIntercept")
-
-  private val coefficientsArray = coefficients match {
--- End diff --

Note this eliminates the check for sparse coefficients, but I'm not sure it 
was ever necessary. We don't do it in other aggregators, this is a private 
class so we don't need to worry about people misusing it. Appreciate other 
thoughts.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16677: [SPARK-19355][SQL] Use map output statistices to ...

2017-02-26 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/16677#discussion_r103138824
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala ---
@@ -90,25 +95,101 @@ trait BaseLimitExec extends UnaryExecNode with 
CodegenSupport {
 }
 
 /**
- * Take the first `limit` elements of each child partition, but do not 
collect or shuffle them.
+ * Take the `limit` elements of the child output.
  */
-case class LocalLimitExec(limit: Int, child: SparkPlan) extends 
BaseLimitExec {
+case class GlobalLimitExec(limit: Int, child: SparkPlan) extends 
UnaryExecNode {
 
-  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+  override def output: Seq[Attribute] = child.output
 
   override def outputPartitioning: Partitioning = child.outputPartitioning
-}
 
-/**
- * Take the first `limit` elements of the child's single output partition.
- */
-case class GlobalLimitExec(limit: Int, child: SparkPlan) extends 
BaseLimitExec {
+  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+
+  private val serializer: Serializer = new 
UnsafeRowSerializer(child.output.size)
+
+  protected override def doExecute(): RDD[InternalRow] = {
+val childRDD = child.execute()
+val partitioner = LocalPartitioning(child.outputPartitioning,
+  childRDD.getNumPartitions)
+val shuffleDependency = ShuffleExchange.prepareShuffleDependency(
+  childRDD, child.output, partitioner, serializer)
+val numberOfOutput: Seq[Int] = if 
(shuffleDependency.rdd.getNumPartitions != 0) {
+  // submitMapStage does not accept RDD with 0 partition.
+  // So, we will not submit this dependency.
+  val submittedStageFuture = 
sparkContext.submitMapStage(shuffleDependency)
+  submittedStageFuture.get().numberOfOutput.toSeq
+} else {
+  Nil
+}
 
-  override def requiredChildDistribution: List[Distribution] = AllTuples 
:: Nil
+// Try to keep child plan's original data parallelism or not. It is 
enabled by default.
+val respectChildParallelism = sqlContext.conf.enableParallelGlobalLimit
 
-  override def outputPartitioning: Partitioning = child.outputPartitioning
+val shuffled = new ShuffledRowRDD(shuffleDependency)
 
-  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+val sumOfOutput = numberOfOutput.sum
+if (sumOfOutput <= limit) {
+  shuffled
+} else if (!respectChildParallelism) {
+  // This is mainly for tests.
+  // We take the rows of each partition until we reach the required 
limit number.
+  var countForRows = 0
+  val takeAmounts = new mutable.HashMap[Int, Int]()
+  numberOfOutput.zipWithIndex.foreach { case (num, index) =>
+if (countForRows + num < limit) {
+  countForRows += num
+  takeAmounts += ((index, num))
+} else {
+  val toTake = limit - countForRows
+  countForRows += toTake
+  takeAmounts += ((index, toTake))
+}
+  }
+  val broadMap = sparkContext.broadcast(takeAmounts)
+  shuffled.mapPartitionsWithIndexInternal { case (index, iter) =>
+broadMap.value.get(index).map { size =>
+  iter.take(size)
+}.get
+  }
+} else {
+  // We try to distribute the required limit number of rows across all 
child rdd's partitions.
+  var numToReduce = (sumOfOutput - limit)
+  val reduceAmounts = new mutable.HashMap[Int, Int]()
+  val nonEmptyParts = numberOfOutput.filter(_ > 0).size
+  val reducePerPart = numToReduce / nonEmptyParts
+  numberOfOutput.zipWithIndex.foreach { case (num, index) =>
+if (num >= reducePerPart) {
+  numToReduce -= reducePerPart
+  reduceAmounts += ((index, reducePerPart))
+} else {
+  numToReduce -= num
+  reduceAmounts += ((index, num))
+}
+  }
+  while (numToReduce > 0) {
+numberOfOutput.zipWithIndex.foreach { case (num, index) =>
+  val toReduce = if (numToReduce / nonEmptyParts > 0) {
+numToReduce / nonEmptyParts
+  } else {
+numToReduce
+  }
+  if (num - reduceAmounts(index) >= toReduce) {
+reduceAmounts(index) = reduceAmounts(index) + toReduce
+numToReduce -= toReduce
+  } else if (num - reduceAmounts(index) > 0) {
+reduceAmounts(index) = reduceAmounts(index) + 1
+numToReduce -= 1
+  }
+}
+  }
+  val broadMap = sparkContext.broadcast(reduceAmounts)
+  

[GitHub] spark issue #17076: [SPARK-19745][ML] SVCAggregator captures coefficients in...

2017-02-26 Thread sethah
Github user sethah commented on the issue:

https://github.com/apache/spark/pull/17076
  
ping @yanboliang @AnthonyTruchet 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17078: [SPARK-19746][ML] Faster indexing for logistic ag...

2017-02-26 Thread sethah
GitHub user sethah opened a pull request:

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

[SPARK-19746][ML] Faster indexing for logistic aggregator

## What changes were proposed in this pull request?

JIRA: [SPARK-19746](https://issues.apache.org/jira/browse/SPARK-19746)

The following code is inefficient:

scala
val localCoefficients: Vector = bcCoefficients.value

features.foreachActive { (index, value) =>
  val stdValue = value / localFeaturesStd(index)
  var j = 0
  while (j < numClasses) {
margins(j) += localCoefficients(index * numClasses + j) * stdValue
j += 1
  }
}


`localCoefficients(index * numClasses + j)` calls `Vector.apply` which 
creates a new Breeze vector and indexes that. Even if it is not that slow to 
create the object, we will generate a lot of extra garbage that may result in 
longer GC pauses. This is a hot inner loop, so we should optimize wherever 
possible.

## How was this patch tested?

I don't think there's a great way to test this patch. It's purely 
performance related, so unit tests should guarantee that we haven't made any 
unwanted changes. Empirically I observed between 10-40% speedups just running 
short local tests. I suspect the big differences will be seen when large 
data/coefficient sizes have to pause for GC more often. I welcome other ideas 
for testing.


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

$ git pull https://github.com/sethah/spark logistic_agg_indexing

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

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


commit 3bea389f6780e1fd0385fbe26954fa4f59b69e37
Author: sethah 
Date:   2017-02-27T05:22:09Z

better indexing for logistic agg




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17078: [SPARK-19746][ML] Faster indexing for logistic aggregato...

2017-02-26 Thread sethah
Github user sethah commented on the issue:

https://github.com/apache/spark/pull/17078
  
ping @dbtsai @yanboliang 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17077: [SPARK-16931][PYTHON][SQL] Add Python wrapper for...

2017-02-26 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/17077#discussion_r103138278
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -2038,6 +2038,53 @@ def test_BinaryType_serialization(self):
 df = self.spark.createDataFrame(data, schema=schema)
 df.collect()
 
+def test_bucketed_write(self):
+data = [
+(1, "foo", 3.0), (2, "foo", 5.0),
+(3, "bar", -1.0), (4, "bar", 6.0),
+]
+df = self.spark.createDataFrame(data, ["x", "y", "z"])
+
+# Test write with one bucketing column
+df.write.bucketBy(3, 
"x").mode("overwrite").saveAsTable("pyspark_bucket")
+self.assertEqual(
+len([c for c in 
self.spark.catalog.listColumns("pyspark_bucket") if c.name == "x" and 
c.isBucket]),
--- End diff --

Oh, BTW, I assume it exceeds 100 length 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 issue #17077: [SPARK-16931][PYTHON][SQL] Add Python wrapper for bucket...

2017-02-26 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/17077
  
Thanks for cc'ing me. Let me please @davies as he was reviewing it and it 
seems close to be merged and also @holdenk.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17077: [SPARK-16931][PYTHON][SQL] Add Python wrapper for...

2017-02-26 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/17077#discussion_r103138165
  
--- Diff: python/pyspark/sql/readwriter.py ---
@@ -545,6 +545,55 @@ def partitionBy(self, *cols):
 self._jwrite = self._jwrite.partitionBy(_to_seq(self._spark._sc, 
cols))
 return self
 
+@since(2.1)
--- End diff --

Maybe it should be 2.2 :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17077: [SPARK-16931][PYTHON][SQL] Add Python wrapper for...

2017-02-26 Thread zero323
GitHub user zero323 opened a pull request:

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

[SPARK-16931][PYTHON][SQL] Add Python wrapper for bucketBy 

## What changes were proposed in this pull request?

Adds Python wrappers for `DataFrameWriter.bucketBy` and 
`DataFrameWriter.sortBy` 
([SPARK-16931](https://issues.apache.org/jira/browse/SPARK-16931))

## How was this patch tested?

Unit tests covering new feature.

__Note__: Based on work of @GregBowyer 
(f49b9a23468f7af32cb53d2b654272757c151725)

CC @HyukjinKwon 

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

$ git pull https://github.com/zero323/spark SPARK-16931

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

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


commit 3024d1cca60793f9aca2cf144bc630106374131c
Author: Greg Bowyer 
Date:   2016-08-06T00:53:30Z

[SPARK-16931][PYTHON] PySpark APIS for bucketBy and sortBy

commit 7d911c647f21ada7fb429fd7c1c5f15934ff8847
Author: zero323 
Date:   2017-02-27T02:59:22Z

Add tests for bucketed writes

commit 72c04a3f196da5223ebb44725aa88cffa81036e4
Author: zero323 
Date:   2017-02-27T02:59:52Z

Check input types in sortBy / bucketBy




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17058: Refactored code to remove null representation

2017-02-26 Thread HarshSharma8
Github user HarshSharma8 closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17054: Refactored the code to remove redundency of count...

2017-02-26 Thread HarshSharma8
Github user HarshSharma8 closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17052: [SPARK-19690][SS] Join a streaming DataFrame with a batc...

2017-02-26 Thread uncleGen
Github user uncleGen commented on the issue:

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


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

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



[GitHub] spark pull request #17076: [SPARK-19745][ML] SVCAggregator captures coeffici...

2017-02-26 Thread sethah
GitHub user sethah opened a pull request:

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

[SPARK-19745][ML] SVCAggregator captures coefficients in its closure

## What changes were proposed in this pull request?

JIRA: [SPARK-19745](https://issues.apache.org/jira/browse/SPARK-19745)

Reorganize SVCAggregator to avoid serializing coefficients. This patch also 
makes the gradient array a `lazy val` which will avoid materializing a large 
array on the driver before shipping the class to the executors. This 
improvement stems from https://github.com/apache/spark/pull/16037. Actually, 
probably all ML aggregators can benefit from this. 

We can either: a.) separate the gradient improvement into another patch b.) 
keep what's here _plus_ add the lazy evaluation to all other aggregators in 
this patch or c.) keep it as is.

## How was this patch tested?

This is an interesting question! I don't know of a reasonable way to test 
this right now. Ideally, we could perform an optimization and look at the 
shuffle write data for each task, and we could compare the size to what it we 
know it should be: `numCoefficients * 8 bytes`. Not sure if there is a good way 
to do that right now? We could discuss this here or in another JIRA, but I 
suspect it would be a significant undertaking. For now, I verified through the 
web ui:

**Before**


![image](https://cloud.githubusercontent.com/assets/7275795/23348865/eeb48382-fc62-11e6-8a97-48e262ee02b8.png)


**After**


![image](https://cloud.githubusercontent.com/assets/7275795/23348872/f933fe14-fc62-11e6-8a1d-b5145775457e.png)




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

$ git pull https://github.com/sethah/spark svc_agg

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

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


commit e87e3347042dbe1a6beb2fed2213da7e10a8abd9
Author: sethah 
Date:   2017-02-27T04:25:14Z

performance cleanup in svc agg




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #14517: [SPARK-16931][PYTHON] PySpark APIS for bucketBy a...

2017-02-26 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/14517#discussion_r103135331
  
--- Diff: python/pyspark/sql/readwriter.py ---
@@ -747,16 +800,25 @@ def _test():
 except py4j.protocol.Py4JError:
 spark = SparkSession(sc)
 
+seed = int(time() * 1000)
--- End diff --

@zero323, Good to know. Then, please go ahead if you are ready :).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #14517: [SPARK-16931][PYTHON] PySpark APIS for bucketBy a...

2017-02-26 Thread zero323
Github user zero323 commented on a diff in the pull request:

https://github.com/apache/spark/pull/14517#discussion_r103134846
  
--- Diff: python/pyspark/sql/readwriter.py ---
@@ -747,16 +800,25 @@ def _test():
 except py4j.protocol.Py4JError:
 spark = SparkSession(sc)
 
+seed = int(time() * 1000)
--- End diff --

@HyukjinKwon By all means. I prepared a bunch of tests 
(7d911c647f21ada7fb429fd7c1c5f15934ff8847)  and extended a bit code provided by 
@GregBowyer (72c04a3f196da5223ebb44725aa88cffa81036e4). I think we can skip low 
level tests (direct access to the files) which are already present in Scala 
test base.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16677: [SPARK-19355][SQL] Use map output statistices to ...

2017-02-26 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/16677#discussion_r103134610
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala ---
@@ -90,25 +95,101 @@ trait BaseLimitExec extends UnaryExecNode with 
CodegenSupport {
 }
 
 /**
- * Take the first `limit` elements of each child partition, but do not 
collect or shuffle them.
+ * Take the `limit` elements of the child output.
  */
-case class LocalLimitExec(limit: Int, child: SparkPlan) extends 
BaseLimitExec {
+case class GlobalLimitExec(limit: Int, child: SparkPlan) extends 
UnaryExecNode {
 
-  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+  override def output: Seq[Attribute] = child.output
 
   override def outputPartitioning: Partitioning = child.outputPartitioning
-}
 
-/**
- * Take the first `limit` elements of the child's single output partition.
- */
-case class GlobalLimitExec(limit: Int, child: SparkPlan) extends 
BaseLimitExec {
+  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+
+  private val serializer: Serializer = new 
UnsafeRowSerializer(child.output.size)
+
+  protected override def doExecute(): RDD[InternalRow] = {
+val childRDD = child.execute()
+val partitioner = LocalPartitioning(child.outputPartitioning,
+  childRDD.getNumPartitions)
+val shuffleDependency = ShuffleExchange.prepareShuffleDependency(
+  childRDD, child.output, partitioner, serializer)
+val numberOfOutput: Seq[Int] = if 
(shuffleDependency.rdd.getNumPartitions != 0) {
+  // submitMapStage does not accept RDD with 0 partition.
+  // So, we will not submit this dependency.
+  val submittedStageFuture = 
sparkContext.submitMapStage(shuffleDependency)
+  submittedStageFuture.get().numberOfOutput.toSeq
+} else {
+  Nil
+}
 
-  override def requiredChildDistribution: List[Distribution] = AllTuples 
:: Nil
+// Try to keep child plan's original data parallelism or not. It is 
enabled by default.
+val respectChildParallelism = sqlContext.conf.enableParallelGlobalLimit
 
-  override def outputPartitioning: Partitioning = child.outputPartitioning
+val shuffled = new ShuffledRowRDD(shuffleDependency)
 
-  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+val sumOfOutput = numberOfOutput.sum
+if (sumOfOutput <= limit) {
+  shuffled
+} else if (!respectChildParallelism) {
+  // This is mainly for tests.
+  // We take the rows of each partition until we reach the required 
limit number.
+  var countForRows = 0
+  val takeAmounts = new mutable.HashMap[Int, Int]()
+  numberOfOutput.zipWithIndex.foreach { case (num, index) =>
+if (countForRows + num < limit) {
+  countForRows += num
+  takeAmounts += ((index, num))
+} else {
+  val toTake = limit - countForRows
+  countForRows += toTake
+  takeAmounts += ((index, toTake))
+}
+  }
+  val broadMap = sparkContext.broadcast(takeAmounts)
+  shuffled.mapPartitionsWithIndexInternal { case (index, iter) =>
+broadMap.value.get(index).map { size =>
+  iter.take(size)
+}.get
+  }
+} else {
+  // We try to distribute the required limit number of rows across all 
child rdd's partitions.
+  var numToReduce = (sumOfOutput - limit)
+  val reduceAmounts = new mutable.HashMap[Int, Int]()
+  val nonEmptyParts = numberOfOutput.filter(_ > 0).size
+  val reducePerPart = numToReduce / nonEmptyParts
+  numberOfOutput.zipWithIndex.foreach { case (num, index) =>
+if (num >= reducePerPart) {
+  numToReduce -= reducePerPart
+  reduceAmounts += ((index, reducePerPart))
+} else {
+  numToReduce -= num
+  reduceAmounts += ((index, num))
+}
+  }
+  while (numToReduce > 0) {
+numberOfOutput.zipWithIndex.foreach { case (num, index) =>
+  val toReduce = if (numToReduce / nonEmptyParts > 0) {
+numToReduce / nonEmptyParts
+  } else {
+numToReduce
+  }
+  if (num - reduceAmounts(index) >= toReduce) {
+reduceAmounts(index) = reduceAmounts(index) + toReduce
+numToReduce -= toReduce
+  } else if (num - reduceAmounts(index) > 0) {
+reduceAmounts(index) = reduceAmounts(index) + 1
+numToReduce -= 1
+  }
+}
+  }
+  val broadMap = sparkContext.broadcast(reduceAmounts)
+  

[GitHub] spark pull request #16910: [SPARK-19575][SQL]Reading from or writing to a hi...

2017-02-26 Thread windpiger
Github user windpiger commented on a diff in the pull request:

https://github.com/apache/spark/pull/16910#discussion_r103134443
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala ---
@@ -114,22 +114,30 @@ class HadoopTableReader(
 val tablePath = hiveTable.getPath
 val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt)
 
-// logDebug("Table input: %s".format(tablePath))
-val ifc = hiveTable.getInputFormatClass
-  .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]]
-val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc)
-
-val attrsWithIndex = attributes.zipWithIndex
-val mutableRow = new SpecificInternalRow(attributes.map(_.dataType))
-
-val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter =>
-  val hconf = broadcastedHadoopConf.value.value
-  val deserializer = deserializerClass.newInstance()
-  deserializer.initialize(hconf, tableDesc.getProperties)
-  HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, 
mutableRow, deserializer)
-}
+val locationPath = new Path(inputPathStr)
+val fs = locationPath.getFileSystem(broadcastedHadoopConf.value.value)
+
+// if the table location does not exist, return an empty RDD
+if (!fs.exists(locationPath)) {
--- End diff --

good catch! Thanks!
I test it in Hive, when the table created by `stored by`(e.g. HBase), even 
if there is a table path created under warehouse path when we create the table, 
but there is no data files exist after we insert into the table, and it is ok 
to select data after we delete the table 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 issue #16966: [SPARK-18409][ML]LSH approxNearestNeighbors should use a...

2017-02-26 Thread Yunni
Github user Yunni commented on the issue:

https://github.com/apache/spark/pull/16966
  
@MLnick I did some experiments with WEX datasets. I have put the results in 
the description.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17064: [SPARK-19736][SQL] refreshByPath should clear all cached...

2017-02-26 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/17064
  
cc @cloud-fan 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16476: [SPARK-19084][SQL] Implement expression field

2017-02-26 Thread gczsjdy
Github user gczsjdy commented on the issue:

https://github.com/apache/spark/pull/16476
  
@tejasapatil Sorry to bother, is there still something need to modify?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16965: [Spark-18450][ML] Scala API Change for LSH AND-amplifica...

2017-02-26 Thread Yunni
Github user Yunni commented on the issue:

https://github.com/apache/spark/pull/16965
  
The number of rows would be O(LN). The memory usage will be different as 
the size of each row has changed before and after the explode. Also the 
Catalyst Optimizer may do projections during join which can also change the 
size of each row.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17075: [SPARK-19727][SQL] Fix for round function that modifies ...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17075
  
Can one of the admins verify this patch?


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

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



[GitHub] spark pull request #17075: [SPARK-19727][SQL] Fix for round function that mo...

2017-02-26 Thread wojtek-szymanski
GitHub user wojtek-szymanski opened a pull request:

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

[SPARK-19727][SQL] Fix for round function that modifies original column

## What changes were proposed in this pull request?

Fix for SQL round function that modifies original column when underlying 
data frame is created from a local product.

import org.apache.spark.sql.functions._

case class NumericRow(value: BigDecimal)

val df = 
spark.createDataFrame(Seq(NumericRow(BigDecimal("1.23456789"

df.show()
++
|   value|
++
|1.2345678900|
++

df.withColumn("value_rounded", round('value)).show()
++-+
|   value|value_rounded|
++-+
|1.00|1|
++-+

## How was this patch tested?

New unit test added to existing suite 
`org.apache.spark.sql.MathFunctionsSuite`

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

$ git pull https://github.com/wojtek-szymanski/spark SPARK-19727

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

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


commit ea7c2d3d1b8d71bacc905d01ca327f465ca61c86
Author: Wojtek Szymanski 
Date:   2017-02-27T01:42:37Z

Fix for SQL round function modifying original column




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16879: [SPARK-19541][SQL] High Availability support for ...

2017-02-26 Thread lvdongr
Github user lvdongr closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #15770: [SPARK-15784][ML]:Add Power Iteration Clustering to spar...

2017-02-26 Thread wangmiao1981
Github user wangmiao1981 commented on the issue:

https://github.com/apache/spark/pull/15770
  
@jkbradley Thanks for your reply! I quickly go through your suggestions. If 
I understand correctly, you prefer making it a `Transformer`, as we previously 
discussed, but changing the input data to fit into to the pipeline. Right? Let 
me think about details and evaluate each options before making the changes. 
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 issue #17024: [SPARK-19525][CORE] Compressing checkpoints.

2017-02-26 Thread mridulm
Github user mridulm commented on the issue:

https://github.com/apache/spark/pull/17024
  
I wonder if adding an extension (to the file) helps based on codec ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17024: [SPARK-19525][CORE] Compressing checkpoints.

2017-02-26 Thread mridulm
Github user mridulm commented on the issue:

https://github.com/apache/spark/pull/17024
  
@aramesh117 looks good !
I would also like someone working on streaming to chime in - since that is 
a common usecase for checkpoint.

+CC @tdas, @zsxwing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16987: [SPARK-19633][SS] FileSource read from FileSink

2017-02-26 Thread lw-lin
Github user lw-lin commented on the issue:

https://github.com/apache/spark/pull/16987
  
Rebased to master and tests updated. @zsxwing would you take another look 
when you've got a minute?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16697: [SPARK-19358][CORE] LiveListenerBus shall log the event ...

2017-02-26 Thread mridulm
Github user mridulm commented on the issue:

https://github.com/apache/spark/pull/16697
  
I probably did not convey my idea properly ...
What I meant was, the only change required be :
`logTrace(s"Dropping events $event")` in `def onDropEvent(event: 
SparkListenerEvent): Unit`
There is no need to aggregate, etc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #14517: [SPARK-16931][PYTHON] PySpark APIS for bucketBy a...

2017-02-26 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/14517#discussion_r103122180
  
--- Diff: python/pyspark/sql/readwriter.py ---
@@ -747,16 +800,25 @@ def _test():
 except py4j.protocol.Py4JError:
 spark = SparkSession(sc)
 
+seed = int(time() * 1000)
--- End diff --

cc @zero323, would you maybe be interested in taking over this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16991: [SPARK-19594][Structured Streaming] StreamingQuer...

2017-02-26 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16991: [SPARK-19594][Structured Streaming] StreamingQueryListen...

2017-02-26 Thread zsxwing
Github user zsxwing commented on the issue:

https://github.com/apache/spark/pull/16991
  
LGTM. Merging to master and 2.1. 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 issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16944
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73497/
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 #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16944
  
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 #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16944
  
**[Test build #73497 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73497/testReport)**
 for PR 16944 at commit 
[`b8d72b4`](https://github.com/apache/spark/commit/b8d72b467e01880c0870a553e27dd0ac771e5489).
 * 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 #16991: [SPARK-19594][Structured Streaming] StreamingQueryListen...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16991
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73496/
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 #16991: [SPARK-19594][Structured Streaming] StreamingQueryListen...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16991
  
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 #16991: [SPARK-19594][Structured Streaming] StreamingQueryListen...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16991
  
**[Test build #73496 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73496/testReport)**
 for PR 16991 at commit 
[`896f5b3`](https://github.com/apache/spark/commit/896f5b33d01fa51db9d8ee53a8cc95274654f9e8).
 * 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 #16782: [SPARK-19348][PYTHON][WIP] PySpark keyword_only decorato...

2017-02-26 Thread jkbradley
Github user jkbradley commented on the issue:

https://github.com/apache/spark/pull/16782
  
Thanks @BryanCutler for the patch!  The fix looks reasonable to me, but let 
me try to check with @davies to confirm.

If this is the right approach, then I think we should update the other uses 
of _input_kwargs in pyspark.ml as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

2017-02-26 Thread budde
Github user budde commented on the issue:

https://github.com/apache/spark/pull/16944
  
Thanks for the thorough feedback.

In the latest update I refactored ```convertToLogicalRelation()``` to 
update the schema of the ```catalogTable``` field of the value it returns 
whenever a schema is successfully inferred rather than only when one is 
inferred and successfully saved. This allowed me to clean up the changes to 
this method a bit. @ericl -- feel free to give this another look over.

Per @cloud-fan's feedback, I added an ```alterTableSchema()``` method to 
the ExternalTable interface. I've added implementations of it to 
```InMemoryCatalog``` and ```HiveExternalCatalog``` along with tests. I've also 
addressed the feedback from @viirya 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16944: [SPARK-19611][SQL] Introduce configurable table s...

2017-02-26 Thread budde
Github user budde commented on a diff in the pull request:

https://github.com/apache/spark/pull/16944#discussion_r103116512
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -161,22 +164,51 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   bucketSpec,
   Some(partitionSchema))
 
+val catalogTable = metastoreRelation.catalogTable
 val logicalRelation = cached.getOrElse {
   val sizeInBytes =
 
metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
   val fileIndex = {
-val index = new CatalogFileIndex(
-  sparkSession, metastoreRelation.catalogTable, sizeInBytes)
+val index = new CatalogFileIndex(sparkSession, catalogTable, 
sizeInBytes)
 if (lazyPruningEnabled) {
   index
 } else {
   index.filterPartitions(Nil)  // materialize all the 
partitions in memory
 }
   }
   val partitionSchemaColumnNames = 
partitionSchema.map(_.name.toLowerCase).toSet
-  val dataSchema =
-StructType(metastoreSchema
-  .filterNot(field => 
partitionSchemaColumnNames.contains(field.name.toLowerCase)))
+  val filteredMetastoreSchema = StructType(metastoreSchema
+.filterNot(field => 
partitionSchemaColumnNames.contains(field.name.toLowerCase)))
+
+  val inferenceMode = 
sparkSession.sessionState.conf.caseSensitiveInferenceMode
+  val shouldInfer = (inferenceMode != NEVER_INFER) && 
!catalogTable.schemaPreservesCase
+
+  val (dataSchema, updatedTable) = if (shouldInfer) {
+logInfo(s"Inferring case-sensitive schema for table $tableName 
(inference mode: " +
+  s"$inferenceMode)")
+val inferredSchema
+  = inferSchema(metastoreSchema, options, defaultSource, 
fileType, fileIndex)
+
+// If configured, save the inferred case-sensitive schema to 
the table properties and
+// fetch the updated CatalogTable record for use in the 
LogicalRelation.
+val updatedTable = if (inferenceMode == INFER_AND_SAVE) {
+  updateCatalogTable(catalogTable, inferredSchema).getOrElse {
+logWarning(s"Unable to save case-sensitive schema for 
table $tableName " +
+  s"(inference mode: $inferenceMode)")
+catalogTable
+  }
+} else {
+  catalogTable
+}
+val schema = inferredSchema.getOrElse {
--- End diff --

See my latest comment at line 281. I'll rework this section.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16944: [SPARK-19611][SQL] Introduce configurable table s...

2017-02-26 Thread budde
Github user budde commented on a diff in the pull request:

https://github.com/apache/spark/pull/16944#discussion_r103116519
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
 ---
@@ -0,0 +1,200 @@
+/*
+ * 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.sql.hive
+
+import java.io.File
+import java.util.concurrent.{Executors, TimeUnit}
+
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog._
+import org.apache.spark.sql.execution.datasources.FileStatusCache
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.hive.client.HiveClient
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
+import org.apache.spark.sql.test.SQLTestUtils
+import org.apache.spark.sql.types._
+
+class HiveSchemaInferenceSuite
+  extends QueryTest with TestHiveSingleton with SQLTestUtils with 
BeforeAndAfterEach {
+
+  import HiveSchemaInferenceSuite._
+  import HiveExternalCatalog.SPARK_SQL_PREFIX
+
+  override def beforeEach(): Unit = {
+super.beforeEach()
+FileStatusCache.resetForTesting()
+  }
+
+  override def afterEach(): Unit = {
+super.afterEach()
+FileStatusCache.resetForTesting()
+  }
+
+  private val externalCatalog = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
+  private val lowercaseSchema = StructType(Seq(
+StructField("fieldone", LongType),
+StructField("partcol1", IntegerType),
+StructField("partcol2", IntegerType)))
+  private val caseSensitiveSchema = StructType(Seq(
+StructField("fieldOne", LongType),
+// Partition columns remain case-insensitive
+StructField("partcol1", IntegerType),
+StructField("partcol2", IntegerType)))
+
+  // Create a CatalogTable instance modeling an external Hive Metastore 
table backed by
+  // Parquet data files.
+  private def hiveExternalCatalogTable(
+  tableName: String,
+  location: String,
+  schema: StructType,
+  partitionColumns: Seq[String],
+  properties: Map[String, String] = Map.empty): CatalogTable = {
+CatalogTable(
+  identifier = TableIdentifier(table = tableName, database = 
Option(DATABASE)),
+  tableType = CatalogTableType.EXTERNAL,
+  storage = CatalogStorageFormat(
+locationUri = Option(location),
+inputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+outputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
+serde = 
Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
+compressed = false,
+properties = Map("serialization.format" -> "1")),
+  schema = schema,
+  provider = Option("hive"),
+  partitionColumnNames = partitionColumns,
+  properties = properties)
+  }
+
+  // Creates CatalogTablePartition instances for adding partitions of data 
to our test table.
+  private def hiveCatalogPartition(location: String, index: Int): 
CatalogTablePartition
+= CatalogTablePartition(
+  spec = Map("partcol1" -> index.toString, "partcol2" -> 
index.toString),
+  storage = CatalogStorageFormat(
+locationUri = 
Option(s"${location}/partCol1=$index/partCol2=$index/"),
+inputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+outputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
+serde = 
Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
+compressed = false,
+properties = 

[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

2017-02-26 Thread budde
Github user budde commented on a diff in the pull request:

https://github.com/apache/spark/pull/16944#discussion_r103116521
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
 ---
@@ -0,0 +1,200 @@
+/*
+ * 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.sql.hive
+
+import java.io.File
+import java.util.concurrent.{Executors, TimeUnit}
+
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog._
+import org.apache.spark.sql.execution.datasources.FileStatusCache
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.hive.client.HiveClient
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
+import org.apache.spark.sql.test.SQLTestUtils
+import org.apache.spark.sql.types._
+
+class HiveSchemaInferenceSuite
+  extends QueryTest with TestHiveSingleton with SQLTestUtils with 
BeforeAndAfterEach {
+
+  import HiveSchemaInferenceSuite._
+  import HiveExternalCatalog.SPARK_SQL_PREFIX
+
+  override def beforeEach(): Unit = {
+super.beforeEach()
+FileStatusCache.resetForTesting()
+  }
+
+  override def afterEach(): Unit = {
+super.afterEach()
+FileStatusCache.resetForTesting()
+  }
+
+  private val externalCatalog = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
+  private val lowercaseSchema = StructType(Seq(
+StructField("fieldone", LongType),
+StructField("partcol1", IntegerType),
+StructField("partcol2", IntegerType)))
+  private val caseSensitiveSchema = StructType(Seq(
+StructField("fieldOne", LongType),
+// Partition columns remain case-insensitive
+StructField("partcol1", IntegerType),
+StructField("partcol2", IntegerType)))
+
+  // Create a CatalogTable instance modeling an external Hive Metastore 
table backed by
+  // Parquet data files.
+  private def hiveExternalCatalogTable(
+  tableName: String,
+  location: String,
+  schema: StructType,
+  partitionColumns: Seq[String],
+  properties: Map[String, String] = Map.empty): CatalogTable = {
+CatalogTable(
+  identifier = TableIdentifier(table = tableName, database = 
Option(DATABASE)),
+  tableType = CatalogTableType.EXTERNAL,
+  storage = CatalogStorageFormat(
+locationUri = Option(location),
+inputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+outputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
+serde = 
Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
+compressed = false,
+properties = Map("serialization.format" -> "1")),
+  schema = schema,
+  provider = Option("hive"),
+  partitionColumnNames = partitionColumns,
+  properties = properties)
+  }
+
+  // Creates CatalogTablePartition instances for adding partitions of data 
to our test table.
+  private def hiveCatalogPartition(location: String, index: Int): 
CatalogTablePartition
+= CatalogTablePartition(
+  spec = Map("partcol1" -> index.toString, "partcol2" -> 
index.toString),
+  storage = CatalogStorageFormat(
+locationUri = 
Option(s"${location}/partCol1=$index/partCol2=$index/"),
+inputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+outputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
+serde = 
Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
+compressed = false,
+properties = 

[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

2017-02-26 Thread budde
Github user budde commented on a diff in the pull request:

https://github.com/apache/spark/pull/16944#discussion_r103116516
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
 ---
@@ -0,0 +1,200 @@
+/*
+ * 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.sql.hive
+
+import java.io.File
+import java.util.concurrent.{Executors, TimeUnit}
+
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog._
+import org.apache.spark.sql.execution.datasources.FileStatusCache
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.hive.client.HiveClient
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
+import org.apache.spark.sql.test.SQLTestUtils
+import org.apache.spark.sql.types._
+
+class HiveSchemaInferenceSuite
+  extends QueryTest with TestHiveSingleton with SQLTestUtils with 
BeforeAndAfterEach {
+
+  import HiveSchemaInferenceSuite._
+  import HiveExternalCatalog.SPARK_SQL_PREFIX
+
+  override def beforeEach(): Unit = {
+super.beforeEach()
+FileStatusCache.resetForTesting()
+  }
+
+  override def afterEach(): Unit = {
+super.afterEach()
+FileStatusCache.resetForTesting()
+  }
+
+  private val externalCatalog = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
+  private val lowercaseSchema = StructType(Seq(
+StructField("fieldone", LongType),
+StructField("partcol1", IntegerType),
+StructField("partcol2", IntegerType)))
+  private val caseSensitiveSchema = StructType(Seq(
+StructField("fieldOne", LongType),
+// Partition columns remain case-insensitive
+StructField("partcol1", IntegerType),
+StructField("partcol2", IntegerType)))
+
+  // Create a CatalogTable instance modeling an external Hive Metastore 
table backed by
+  // Parquet data files.
+  private def hiveExternalCatalogTable(
+  tableName: String,
+  location: String,
+  schema: StructType,
+  partitionColumns: Seq[String],
+  properties: Map[String, String] = Map.empty): CatalogTable = {
+CatalogTable(
+  identifier = TableIdentifier(table = tableName, database = 
Option(DATABASE)),
+  tableType = CatalogTableType.EXTERNAL,
+  storage = CatalogStorageFormat(
+locationUri = Option(location),
+inputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+outputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
+serde = 
Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
+compressed = false,
+properties = Map("serialization.format" -> "1")),
+  schema = schema,
+  provider = Option("hive"),
+  partitionColumnNames = partitionColumns,
+  properties = properties)
+  }
+
+  // Creates CatalogTablePartition instances for adding partitions of data 
to our test table.
+  private def hiveCatalogPartition(location: String, index: Int): 
CatalogTablePartition
+= CatalogTablePartition(
+  spec = Map("partcol1" -> index.toString, "partcol2" -> 
index.toString),
+  storage = CatalogStorageFormat(
+locationUri = 
Option(s"${location}/partCol1=$index/partCol2=$index/"),
+inputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+outputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
+serde = 
Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
+compressed = false,
+properties = 

[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

2017-02-26 Thread budde
Github user budde commented on a diff in the pull request:

https://github.com/apache/spark/pull/16944#discussion_r103116517
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala
 ---
@@ -0,0 +1,200 @@
+/*
+ * 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.sql.hive
+
+import java.io.File
+import java.util.concurrent.{Executors, TimeUnit}
+
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog._
+import org.apache.spark.sql.execution.datasources.FileStatusCache
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.hive.client.HiveClient
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
+import org.apache.spark.sql.test.SQLTestUtils
+import org.apache.spark.sql.types._
+
+class HiveSchemaInferenceSuite
+  extends QueryTest with TestHiveSingleton with SQLTestUtils with 
BeforeAndAfterEach {
+
+  import HiveSchemaInferenceSuite._
+  import HiveExternalCatalog.SPARK_SQL_PREFIX
+
+  override def beforeEach(): Unit = {
+super.beforeEach()
+FileStatusCache.resetForTesting()
+  }
+
+  override def afterEach(): Unit = {
+super.afterEach()
+FileStatusCache.resetForTesting()
+  }
+
+  private val externalCatalog = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
+  private val lowercaseSchema = StructType(Seq(
+StructField("fieldone", LongType),
+StructField("partcol1", IntegerType),
+StructField("partcol2", IntegerType)))
+  private val caseSensitiveSchema = StructType(Seq(
+StructField("fieldOne", LongType),
+// Partition columns remain case-insensitive
+StructField("partcol1", IntegerType),
+StructField("partcol2", IntegerType)))
+
+  // Create a CatalogTable instance modeling an external Hive Metastore 
table backed by
+  // Parquet data files.
+  private def hiveExternalCatalogTable(
+  tableName: String,
+  location: String,
+  schema: StructType,
+  partitionColumns: Seq[String],
+  properties: Map[String, String] = Map.empty): CatalogTable = {
+CatalogTable(
+  identifier = TableIdentifier(table = tableName, database = 
Option(DATABASE)),
+  tableType = CatalogTableType.EXTERNAL,
+  storage = CatalogStorageFormat(
+locationUri = Option(location),
+inputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+outputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
+serde = 
Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
+compressed = false,
+properties = Map("serialization.format" -> "1")),
+  schema = schema,
+  provider = Option("hive"),
+  partitionColumnNames = partitionColumns,
+  properties = properties)
+  }
+
+  // Creates CatalogTablePartition instances for adding partitions of data 
to our test table.
+  private def hiveCatalogPartition(location: String, index: Int): 
CatalogTablePartition
+= CatalogTablePartition(
+  spec = Map("partcol1" -> index.toString, "partcol2" -> 
index.toString),
+  storage = CatalogStorageFormat(
+locationUri = 
Option(s"${location}/partCol1=$index/partCol2=$index/"),
+inputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+outputFormat = 
Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
+serde = 
Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
+compressed = false,
+properties = 

[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

2017-02-26 Thread budde
Github user budde commented on a diff in the pull request:

https://github.com/apache/spark/pull/16944#discussion_r103116509
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -226,6 +258,41 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
 result.copy(expectedOutputAttributes = Some(metastoreRelation.output))
   }
 
+  private def inferSchema(
+  metastoreSchema: StructType,
+  options: Map[String, String],
+  fileFormat: FileFormat,
+  fileType: String,
+  fileIndex: FileIndex): Option[StructType] = {
+val inferred = fileFormat.inferSchema(
+  sparkSession,
+  options,
+  fileIndex.listFiles(Nil).flatMap(_.files))
+if (fileType.equals("parquet")) {
+  
inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
+} else {
+  inferred
+}
+  }
+
+  private def updateCatalogTable(
+  catalogTable: CatalogTable,
+  inferredSchema: Option[StructType]): Option[CatalogTable] = try {
+inferredSchema.flatMap { schema =>
--- End diff --

Actually, thinking about it, I think we'll want to update 
```catalogTable.schema ``` any time we successfully infer a schema. This means 
this function can be converted to return ```Unit``` and a lot of the changes to 
```convertToLogicalRelation``` can be simplified


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16782: [SPARK-19348][PYTHON][WIP] PySpark keyword_only decorato...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16782
  
**[Test build #3586 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3586/testReport)**
 for PR 16782 at commit 
[`83bcce0`](https://github.com/apache/spark/commit/83bcce0c2fc33184579224fca18066dbf6891d26).
 * 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 #16782: [SPARK-19348][PYTHON][WIP] PySpark keyword_only decorato...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16782
  
**[Test build #3586 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3586/testReport)**
 for PR 16782 at commit 
[`83bcce0`](https://github.com/apache/spark/commit/83bcce0c2fc33184579224fca18066dbf6891d26).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16697: [SPARK-19358][CORE] LiveListenerBus shall log the event ...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16697
  
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 #16991: [SPARK-19594][Structured Streaming] StreamingQueryListen...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16991
  
**[Test build #73496 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73496/testReport)**
 for PR 16991 at commit 
[`896f5b3`](https://github.com/apache/spark/commit/896f5b33d01fa51db9d8ee53a8cc95274654f9e8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16697: [SPARK-19358][CORE] LiveListenerBus shall log the event ...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16697
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73495/
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 #16697: [SPARK-19358][CORE] LiveListenerBus shall log the event ...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16697
  
**[Test build #73495 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73495/testReport)**
 for PR 16697 at commit 
[`673b97c`](https://github.com/apache/spark/commit/673b97c9bb32e52cedc41a92dfed7ddd5f70d2b6).
 * 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 #16991: [SPARK-19594][Structured Streaming] StreamingQueryListen...

2017-02-26 Thread zsxwing
Github user zsxwing commented on the issue:

https://github.com/apache/spark/pull/16991
  
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 issue #17067: [SPARK-19602][SQL][TESTS] Add tests for qualified column...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17067
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73494/
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 #17067: [SPARK-19602][SQL][TESTS] Add tests for qualified column...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17067
  
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 #17067: [SPARK-19602][SQL][TESTS] Add tests for qualified column...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17067
  
**[Test build #73494 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73494/testReport)**
 for PR 17067 at commit 
[`2f9937e`](https://github.com/apache/spark/commit/2f9937e5dc0fc02218c644083432f1bff241409f).
 * 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 #17060: [SQL] Duplicate test exception in SQLQueryTestSuite due ...

2017-02-26 Thread dilipbiswal
Github user dilipbiswal commented on the issue:

https://github.com/apache/spark/pull/17060
  
Thank you @srowen @gatorsmile 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17059: [SPARK-19733][ML]Removed unnecessary castings and refact...

2017-02-26 Thread datumbox
Github user datumbox commented on the issue:

https://github.com/apache/spark/pull/17059
  
I decided to provide a few more bencharks in order to alleviate some of the 
concerns raised by @srowen. 

To reproduce the results add the following snippet in the ALSSuite class:
```scala
  test("Speed difference") {
val (training, test) =
  genExplicitTestData(numUsers = 200, numItems = 400, rank = 2, 
noiseStd = 0.01)

val runs = 1000
var totalTime = 0.0
println("Performing "+runs+" runs")
println("Run Id,Time (secs)")
for(i <- 0 until runs) {
  val t0 = System.currentTimeMillis
  testALS(training, test, maxIter = 1, rank = 2, regParam = 0.01, 
targetRMSE = 0.1)
  val secs = (System.currentTimeMillis - t0)/1000.0
  println(i+","+secs)
  totalTime += secs
}
println("AVG Execution Time: "+(totalTime/runs)+" secs")
  }
```
To test both solutions, I collected 1000 samples for each (took ~1 hour for 
each). Here you can see the detailed output for the 
[original](http://pastebin.com/ys9Vejs9) and the 
[proposed](http://pastebin.com/dCpkyMGc) code. 

| Code | Mean Execution Time | Std |
| --- | --- | --- |
| Original | 4.75521 | 0.81237 |
| Proposed | 4.56276 | 0.72790 |

Using an unpaired t-test to compare the two means we find that the proposed 
code is faster and the result is statistically significant (p-value < .0001). 

Below I summarize why I believe the original code needs to change:
1. Casting user and item ids into double and then to integer is a hacky & 
indirect way to validate that the ids are numerical and within integer range. 
The proposed code covers all the corner cases in a clear and direct way. As an 
added bonus, it handles Doubles and Floats with fractional part.
2. Given that the ALS implementation requires ids with int values, it is 
expected that the majority of users encode their Ids as Integer. The proposed 
solution avoids any casting in that case while reducing the casting in all the 
other cases. This avoids putting unnecessary strain on the garbage collector, 
something that you can observe if you profile the execution on a large dataset.
3. The proposed solution is not slower than the original; if something it 
is slightly faster.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17074: [SPARK-18646][REPL] Set parent classloader as null for E...

2017-02-26 Thread taroplus
Github user taroplus commented on the issue:

https://github.com/apache/spark/pull/17074
  
@holdenk 
I believe this was introduced in this change

https://github.com/apache/spark/commit/fa0524fd02eedd0bbf1edc750dc3997a86ea25f5#diff-bb538fda94224dd0af01d0fd7e1b4ea0R39

the change is to stop passing 'parent' to the parent class(ClassLoader), 
however this introduced a link between ExecutorClassLoader to SystemClassLoader.

Most of the time, SystemClassLoader is parent loader itself. Then it causes 
issues with userClassPathFirst (SPARK-18646) and if it is not parent loader, it 
causes issues like InvalidClassException(SPARK-19675)

I added a UT for this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16697: [SPARK-19358][CORE] LiveListenerBus shall log the event ...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16697
  
**[Test build #73495 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73495/testReport)**
 for PR 16697 at commit 
[`673b97c`](https://github.com/apache/spark/commit/673b97c9bb32e52cedc41a92dfed7ddd5f70d2b6).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16697: [SPARK-19358][CORE] LiveListenerBus shall log the event ...

2017-02-26 Thread CodingCat
Github user CodingCat commented on the issue:

https://github.com/apache/spark/pull/16697
  
@mridulm thanks, I have changed the code


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

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17067
  
**[Test build #73494 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73494/testReport)**
 for PR 17067 at commit 
[`2f9937e`](https://github.com/apache/spark/commit/2f9937e5dc0fc02218c644083432f1bff241409f).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17067: [SPARK-19602][SQL][TESTS] Add tests for qualified column...

2017-02-26 Thread gatorsmile
Github user gatorsmile commented on the issue:

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


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

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



[GitHub] spark issue #17074: [SPARK-18646][REPL] Set parent classloader as null for E...

2017-02-26 Thread holdenk
Github user holdenk commented on the issue:

https://github.com/apache/spark/pull/17074
  
Thanks for working on this. It's been awhile since I looked at the 
classloader bits so I'd need to refresh my memory a bit, but regardless this PR 
probably needs better tests - specifically including something which wasn't 
working before and does work with the change.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16976: [SPARK-19610][SQL] Support parsing multiline CSV files

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16976
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73493/
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 #16976: [SPARK-19610][SQL] Support parsing multiline CSV files

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16976
  
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 #16976: [SPARK-19610][SQL] Support parsing multiline CSV files

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16976
  
**[Test build #73493 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73493/testReport)**
 for PR 16976 at commit 
[`22eb29e`](https://github.com/apache/spark/commit/22eb29eddfde484fbd887a32bf8be0b681c4d714).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `abstract class CSVDataSource extends Serializable `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17074: [SPARK-18646][REPL] Set parent classloader as null for E...

2017-02-26 Thread taroplus
Github user taroplus commented on the issue:

https://github.com/apache/spark/pull/17074
  
@holdenk @zsxwing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #17074: [SPARK-18646][REPL] Set parent classloader as null for E...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17074
  
Can one of the admins verify this patch?


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

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



[GitHub] spark pull request #17074: [SPARK-18646][REPL] Set parent classloader as nul...

2017-02-26 Thread taroplus
GitHub user taroplus opened a pull request:

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

[SPARK-18646][REPL] Set parent classloader as null for ExecutorClassLoader

## What changes were proposed in this pull request?

By default, a class loader will have java's system class loader as its 
parent, however this ExecutorClassLoader should always resolve classes from 
given 'paren't class loader.

Problem happens when an application has a class loader structure like below

System-Classloader [ClassA, Runtime]
|
Application-Classloader [ClassA, Spark, Jackson etc etc] : **paren'**
|
Executor Class Loader

The application serializes ClassA using Application class loader however 
Spark deserializes the class using System Class loader, this leads to various 
problems.

The change here to pass null to the parent class so that a link between 
ExecutorClassLoader and SystemClassLoader no longer exists.

For most of situations, the parent class loader is the system class loader 
so this wouldn't have any impact.

## How was this patch tested?

UTs

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

$ git pull https://github.com/taroplus/spark executor_classloader

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

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


commit e0d95381ffc70c22bc2c8a27f908d97f565b2566
Author: Kohki Nishio 
Date:   2017-02-26T16:22:03Z

Set parent classloader as null for ExecutorClassLoader

This change is to address a classloader problem under sbt environment
more details can be found in Jira 
[https://issues.apache.org/jira/browse/SPARK-19675]




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16986: [SPARK-18891][SQL] Support for Map collection types

2017-02-26 Thread michalsenkyr
Github user michalsenkyr commented on the issue:

https://github.com/apache/spark/pull/16986
  
Added support for Java Maps with support for pre-allocation (capacity 
argument on constructor) and sensible defaults for interfaces/abstract classes. 
Also includes implicit encoders.
Updated codegen in description (only a cosmetic change) and added codegen 
for Java Map.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16976: [SPARK-19610][SQL] Support parsing multiline CSV files

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16976
  
**[Test build #73493 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73493/testReport)**
 for PR 16976 at commit 
[`22eb29e`](https://github.com/apache/spark/commit/22eb29eddfde484fbd887a32bf8be0b681c4d714).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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 #16987: [SPARK-19633][SS] FileSource read from FileSink

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16987
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73492/
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 #16987: [SPARK-19633][SS] FileSource read from FileSink

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16987
  
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 #16987: [SPARK-19633][SS] FileSource read from FileSink

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16987
  
**[Test build #73492 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73492/testReport)**
 for PR 16987 at commit 
[`d31cb76`](https://github.com/apache/spark/commit/d31cb76756f7aa2c9c3c803d263ae81f5f509ff2).
 * 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 #17072: [MINOR][BUILD] Fix lint-java breaks in Java

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17072
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73486/
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 #17072: [MINOR][BUILD] Fix lint-java breaks in Java

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17072
  
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 #17072: [MINOR][BUILD] Fix lint-java breaks in Java

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17072
  
**[Test build #73486 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73486/testReport)**
 for PR 17072 at commit 
[`2294a19`](https://github.com/apache/spark/commit/2294a190b93194a39ba7a80c04a40ed0f9abefe4).
 * 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 #17072: [MINOR][BUILD] Fix lint-java breaks in Java

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17072
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73485/
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 #17072: [MINOR][BUILD] Fix lint-java breaks in Java

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17072
  
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 #17072: [MINOR][BUILD] Fix lint-java breaks in Java

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17072
  
**[Test build #73485 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73485/testReport)**
 for PR 17072 at commit 
[`647457c`](https://github.com/apache/spark/commit/647457c5bf75b67624d0b5d2f4440e34d909c845).
 * 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 #16929: [SPARK-19595][SQL] Support json array in from_json

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16929
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73491/
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 #16929: [SPARK-19595][SQL] Support json array in from_json

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16929
  
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 #16929: [SPARK-19595][SQL] Support json array in from_json

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16929
  
**[Test build #73491 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73491/testReport)**
 for PR 16929 at commit 
[`470d879`](https://github.com/apache/spark/commit/470d87969d8fa2de6adfd3765086e03ec8f12252).
 * 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 #17071: [SPARK-15615][SQL][BUILD][FOLLOW-UP] Replace deprecated ...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17071
  
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 #17071: [SPARK-15615][SQL][BUILD][FOLLOW-UP] Replace deprecated ...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17071
  
**[Test build #73489 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73489/testReport)**
 for PR 17071 at commit 
[`aa82df2`](https://github.com/apache/spark/commit/aa82df2e4e106115c8d167aa773af5a46436d4ad).
 * 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 #17071: [SPARK-15615][SQL][BUILD][FOLLOW-UP] Replace deprecated ...

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

2017-02-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17071
  
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 #17071: [SPARK-15615][SQL][BUILD][FOLLOW-UP] Replace deprecated ...

2017-02-26 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17071
  
**[Test build #73488 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73488/testReport)**
 for PR 17071 at commit 
[`2e89259`](https://github.com/apache/spark/commit/2e892592ce644e1d81869224300bc510375c49f5).
 * 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



  1   2   >