[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

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

https://github.com/apache/spark/pull/15102#discussion_r80563543
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,446 @@
+/*
+ * 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.kafka010
+
+import java.{util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, 
KafkaConsumer}
+import 
org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.serialization.ByteArrayDeserializer
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.ExecutorCacheTaskLocation
+import org.apache.spark.sql._
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.kafka010.KafkaSource._
+import org.apache.spark.sql.sources.{DataSourceRegister, 
StreamSourceProvider}
+import org.apache.spark.sql.types._
+import org.apache.spark.SparkContext
+
+/**
+ * A [[Source]] that uses Kafka's own [[KafkaConsumer]] API to reads data 
from Kafka. The design
+ * for this source is as follows.
+ *
+ * - The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this 
source that contains
+ *   a map of TopicPartition -> offset. Note that this offset is 1 + 
(available offset). For
+ *   example if the last record in a Kafka topic "t", partition 2 is 
offset 5, then
+ *   KafkaSourceOffset will contain TopicPartition("t", 2) -> 6. This is 
done keep it consistent
+ *   with the semantics of `KafkaConsumer.position()`.
+ *
+ * - The [[ConsumerStrategy]] class defines which Kafka topics and 
partitions should be read
+ *   by this source. These strategies directly correspond to the different 
consumption options
+ *   in . This class is designed to return a configured
--- End diff --

Still cuts off.  I'd consider removing this abstraction completely.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15235: [SPARK-17661][SQL] Consolidate various listLeafFi...

2016-09-26 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/15235#discussion_r80582937
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 ---
@@ -82,73 +85,185 @@ class ListingFileCatalog(
* This is publicly visible for testing.
*/
   def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = 
{
-if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
-  HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
-} else {
-  // Right now, the number of paths is less than the value of
-  // parallelPartitionDiscoveryThreshold. So, we will list file 
statues at the driver.
-  // If there is any child that has more files than the threshold, we 
will use parallel
-  // listing.
-
-  // Dummy jobconf to get to the pathFilter defined in configuration
-  val jobConf = new JobConf(hadoopConf, this.getClass)
-  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
-
-  val statuses: Seq[FileStatus] = paths.flatMap { path =>
-val fs = path.getFileSystem(hadoopConf)
-logTrace(s"Listing $path on driver")
-
-val childStatuses = {
-  try {
-val stats = fs.listStatus(path)
-if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-  } catch {
-case _: FileNotFoundException =>
-  logWarning(s"The directory $path was not found. Was it 
deleted very recently?")
-  Array.empty[FileStatus]
-  }
-}
+val files =
+  if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
+ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
+  } else {
+ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf)
+  }
+
+mutable.LinkedHashSet(files: _*)
+  }
+
+  override def equals(other: Any): Boolean = other match {
+case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet
+case _ => false
+  }
+
+  override def hashCode(): Int = paths.toSet.hashCode()
+}
+
+
+object ListingFileCatalog extends Logging {
+
+  // `FileStatus` is Writable but not serializable.  What make it worse, 
somehow it doesn't play
+  // well with `SerializableWritable`.  So there seems to be no way to 
serialize a `FileStatus`.
+  // Here we use `SerializableFileStatus` to extract key components of a 
`FileStatus` to serialize
+  // it from executor side and reconstruct it on driver side.
+  private case class SerializableBlockLocation(
+  names: Array[String],
+  hosts: Array[String],
+  offset: Long,
+  length: Long)
+
+  private case class SerializableFileStatus(
+  path: String,
+  length: Long,
+  isDir: Boolean,
+  blockReplication: Short,
+  blockSize: Long,
+  modificationTime: Long,
+  accessTime: Long,
+  blockLocations: Array[SerializableBlockLocation])
+
+  /**
+   * List a collection of path recursively.
+   */
+  private def listLeafFilesInSerial(
+  paths: Seq[Path],
+  hadoopConf: Configuration): Seq[FileStatus] = {
+// Dummy jobconf to get to the pathFilter defined in configuration
+val jobConf = new JobConf(hadoopConf, this.getClass)
+val filter = FileInputFormat.getInputPathFilter(jobConf)
+
+paths.flatMap { path =>
+  logTrace(s"Listing $path")
+  val fs = path.getFileSystem(hadoopConf)
+
+  // [SPARK-17599] Prevent ListingFileCatalog from failing if path 
doesn't exist
+  val status: Option[FileStatus] = try Option(fs.getFileStatus(path)) 
catch {
+case _: FileNotFoundException =>
+  logWarning(s"The directory $path was not found. Was it deleted 
very recently?")
+  None
+  }
 
-childStatuses.map {
-  case f: LocatedFileStatus => f
-
-  // NOTE:
-  //
-  // - Although S3/S3A/S3N file system can be quite slow for 
remote file metadata
-  //   operations, calling `getFileBlockLocations` does no harm 
here since these file system
-  //   implementations don't actually issue RPC for this method.
-  //
-  // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should not
-  //   be a big deal since we always use to 
`listLeafFilesInParallel` when the number of
-  //   paths exceeds threshold.
-  case f =>
-if (f.isDirectory 

[GitHub] spark pull request #15235: [SPARK-17661][SQL] Consolidate various listLeafFi...

2016-09-26 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/15235#discussion_r80583033
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalogSuite.scala
 ---
@@ -0,0 +1,34 @@
+/*
+ * 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.execution.datasources
+
+import org.apache.spark.SparkFunSuite
+
+class ListingFileCatalogSuite extends SparkFunSuite {
+
+  test("file filtering") {
--- End diff --

you may add this test to `FileCatalogSuite` if you like


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15235: [SPARK-17661][SQL] Consolidate various listLeafFi...

2016-09-26 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/15235#discussion_r80582702
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 ---
@@ -82,73 +85,185 @@ class ListingFileCatalog(
* This is publicly visible for testing.
*/
   def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = 
{
-if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
-  HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
-} else {
-  // Right now, the number of paths is less than the value of
-  // parallelPartitionDiscoveryThreshold. So, we will list file 
statues at the driver.
-  // If there is any child that has more files than the threshold, we 
will use parallel
-  // listing.
-
-  // Dummy jobconf to get to the pathFilter defined in configuration
-  val jobConf = new JobConf(hadoopConf, this.getClass)
-  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
-
-  val statuses: Seq[FileStatus] = paths.flatMap { path =>
-val fs = path.getFileSystem(hadoopConf)
-logTrace(s"Listing $path on driver")
-
-val childStatuses = {
-  try {
-val stats = fs.listStatus(path)
-if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-  } catch {
-case _: FileNotFoundException =>
-  logWarning(s"The directory $path was not found. Was it 
deleted very recently?")
-  Array.empty[FileStatus]
-  }
-}
+val files =
+  if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
+ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
+  } else {
+ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf)
+  }
+
+mutable.LinkedHashSet(files: _*)
+  }
+
+  override def equals(other: Any): Boolean = other match {
+case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet
+case _ => false
+  }
+
+  override def hashCode(): Int = paths.toSet.hashCode()
+}
+
+
+object ListingFileCatalog extends Logging {
+
+  // `FileStatus` is Writable but not serializable.  What make it worse, 
somehow it doesn't play
+  // well with `SerializableWritable`.  So there seems to be no way to 
serialize a `FileStatus`.
+  // Here we use `SerializableFileStatus` to extract key components of a 
`FileStatus` to serialize
+  // it from executor side and reconstruct it on driver side.
+  private case class SerializableBlockLocation(
+  names: Array[String],
+  hosts: Array[String],
+  offset: Long,
+  length: Long)
+
+  private case class SerializableFileStatus(
+  path: String,
+  length: Long,
+  isDir: Boolean,
+  blockReplication: Short,
+  blockSize: Long,
+  modificationTime: Long,
+  accessTime: Long,
+  blockLocations: Array[SerializableBlockLocation])
+
+  /**
+   * List a collection of path recursively.
+   */
+  private def listLeafFilesInSerial(
+  paths: Seq[Path],
+  hadoopConf: Configuration): Seq[FileStatus] = {
+// Dummy jobconf to get to the pathFilter defined in configuration
+val jobConf = new JobConf(hadoopConf, this.getClass)
+val filter = FileInputFormat.getInputPathFilter(jobConf)
+
+paths.flatMap { path =>
+  logTrace(s"Listing $path")
+  val fs = path.getFileSystem(hadoopConf)
+
+  // [SPARK-17599] Prevent ListingFileCatalog from failing if path 
doesn't exist
+  val status: Option[FileStatus] = try Option(fs.getFileStatus(path)) 
catch {
+case _: FileNotFoundException =>
+  logWarning(s"The directory $path was not found. Was it deleted 
very recently?")
+  None
+  }
 
-childStatuses.map {
-  case f: LocatedFileStatus => f
-
-  // NOTE:
-  //
-  // - Although S3/S3A/S3N file system can be quite slow for 
remote file metadata
-  //   operations, calling `getFileBlockLocations` does no harm 
here since these file system
-  //   implementations don't actually issue RPC for this method.
-  //
-  // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should not
-  //   be a big deal since we always use to 
`listLeafFilesInParallel` when the number of
-  //   paths exceeds threshold.
-  case f =>
-if (f.isDirectory 

[GitHub] spark pull request #15235: [SPARK-17661][SQL] Consolidate various listLeafFi...

2016-09-26 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/15235#discussion_r80583392
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 ---
@@ -82,73 +83,177 @@ class ListingFileCatalog(
* This is publicly visible for testing.
*/
   def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = 
{
-if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
-  HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
-} else {
-  // Right now, the number of paths is less than the value of
-  // parallelPartitionDiscoveryThreshold. So, we will list file 
statues at the driver.
-  // If there is any child that has more files than the threshold, we 
will use parallel
-  // listing.
-
-  // Dummy jobconf to get to the pathFilter defined in configuration
-  val jobConf = new JobConf(hadoopConf, this.getClass)
-  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
-
-  val statuses: Seq[FileStatus] = paths.flatMap { path =>
-val fs = path.getFileSystem(hadoopConf)
-logTrace(s"Listing $path on driver")
-
-val childStatuses = {
-  try {
-val stats = fs.listStatus(path)
-if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-  } catch {
-case _: FileNotFoundException =>
-  logWarning(s"The directory $path was not found. Was it 
deleted very recently?")
-  Array.empty[FileStatus]
-  }
-}
+val files =
+  if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
+ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
+  } else {
+ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf)
+  }
+
+mutable.LinkedHashSet(files: _*)
+  }
+
+  override def equals(other: Any): Boolean = other match {
+case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet
+case _ => false
+  }
+
+  override def hashCode(): Int = paths.toSet.hashCode()
+}
+
+
+object ListingFileCatalog extends Logging {
 
-childStatuses.map {
-  case f: LocatedFileStatus => f
-
-  // NOTE:
-  //
-  // - Although S3/S3A/S3N file system can be quite slow for 
remote file metadata
-  //   operations, calling `getFileBlockLocations` does no harm 
here since these file system
-  //   implementations don't actually issue RPC for this method.
-  //
-  // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should not
-  //   be a big deal since we always use to 
`listLeafFilesInParallel` when the number of
-  //   paths exceeds threshold.
-  case f =>
-if (f.isDirectory ) {
-  // If f is a directory, we do not need to call 
getFileBlockLocations (SPARK-14959).
-  f
-} else {
-  HadoopFsRelation.createLocatedFileStatus(f, 
fs.getFileBlockLocations(f, 0, f.getLen))
+  // `FileStatus` is Writable but not serializable.  What make it worse, 
somehow it doesn't play
+  // well with `SerializableWritable`.  So there seems to be no way to 
serialize a `FileStatus`.
+  // Here we use `SerializableFileStatus` to extract key components of a 
`FileStatus` to serialize
+  // it from executor side and reconstruct it on driver side.
+  private case class SerializableBlockLocation(
--- End diff --

+1


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

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



[GitHub] spark pull request #15235: [SPARK-17661][SQL] Consolidate various listLeafFi...

2016-09-26 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/15235#discussion_r80573942
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 ---
@@ -82,73 +85,185 @@ class ListingFileCatalog(
* This is publicly visible for testing.
*/
   def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = 
{
-if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
-  HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
-} else {
-  // Right now, the number of paths is less than the value of
-  // parallelPartitionDiscoveryThreshold. So, we will list file 
statues at the driver.
-  // If there is any child that has more files than the threshold, we 
will use parallel
-  // listing.
-
-  // Dummy jobconf to get to the pathFilter defined in configuration
-  val jobConf = new JobConf(hadoopConf, this.getClass)
-  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
-
-  val statuses: Seq[FileStatus] = paths.flatMap { path =>
-val fs = path.getFileSystem(hadoopConf)
-logTrace(s"Listing $path on driver")
-
-val childStatuses = {
-  try {
-val stats = fs.listStatus(path)
-if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-  } catch {
-case _: FileNotFoundException =>
-  logWarning(s"The directory $path was not found. Was it 
deleted very recently?")
-  Array.empty[FileStatus]
-  }
-}
+val files =
+  if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
+ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
+  } else {
+ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf)
+  }
+
+mutable.LinkedHashSet(files: _*)
+  }
+
+  override def equals(other: Any): Boolean = other match {
+case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet
+case _ => false
+  }
+
+  override def hashCode(): Int = paths.toSet.hashCode()
+}
+
+
+object ListingFileCatalog extends Logging {
+
+  // `FileStatus` is Writable but not serializable.  What make it worse, 
somehow it doesn't play
--- End diff --

existing, but good to fix, the comment belongs on the class below not 
`SerializableBlockLocation`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15235: [SPARK-17661][SQL] Consolidate various listLeafFi...

2016-09-26 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/15235#discussion_r80583326
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 ---
@@ -82,73 +85,185 @@ class ListingFileCatalog(
* This is publicly visible for testing.
*/
   def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = 
{
-if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
-  HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
-} else {
-  // Right now, the number of paths is less than the value of
-  // parallelPartitionDiscoveryThreshold. So, we will list file 
statues at the driver.
-  // If there is any child that has more files than the threshold, we 
will use parallel
-  // listing.
-
-  // Dummy jobconf to get to the pathFilter defined in configuration
-  val jobConf = new JobConf(hadoopConf, this.getClass)
-  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
-
-  val statuses: Seq[FileStatus] = paths.flatMap { path =>
-val fs = path.getFileSystem(hadoopConf)
-logTrace(s"Listing $path on driver")
-
-val childStatuses = {
-  try {
-val stats = fs.listStatus(path)
-if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-  } catch {
-case _: FileNotFoundException =>
-  logWarning(s"The directory $path was not found. Was it 
deleted very recently?")
-  Array.empty[FileStatus]
-  }
-}
+val files =
+  if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
+ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
+  } else {
+ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf)
+  }
+
+mutable.LinkedHashSet(files: _*)
+  }
+
+  override def equals(other: Any): Boolean = other match {
+case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet
+case _ => false
+  }
+
+  override def hashCode(): Int = paths.toSet.hashCode()
+}
+
+
+object ListingFileCatalog extends Logging {
+
+  // `FileStatus` is Writable but not serializable.  What make it worse, 
somehow it doesn't play
+  // well with `SerializableWritable`.  So there seems to be no way to 
serialize a `FileStatus`.
+  // Here we use `SerializableFileStatus` to extract key components of a 
`FileStatus` to serialize
+  // it from executor side and reconstruct it on driver side.
+  private case class SerializableBlockLocation(
+  names: Array[String],
+  hosts: Array[String],
+  offset: Long,
+  length: Long)
+
+  private case class SerializableFileStatus(
+  path: String,
+  length: Long,
+  isDir: Boolean,
+  blockReplication: Short,
+  blockSize: Long,
+  modificationTime: Long,
+  accessTime: Long,
+  blockLocations: Array[SerializableBlockLocation])
+
+  /**
+   * List a collection of path recursively.
+   */
+  private def listLeafFilesInSerial(
+  paths: Seq[Path],
+  hadoopConf: Configuration): Seq[FileStatus] = {
+// Dummy jobconf to get to the pathFilter defined in configuration
+val jobConf = new JobConf(hadoopConf, this.getClass)
+val filter = FileInputFormat.getInputPathFilter(jobConf)
+
+paths.flatMap { path =>
+  logTrace(s"Listing $path")
+  val fs = path.getFileSystem(hadoopConf)
+
+  // [SPARK-17599] Prevent ListingFileCatalog from failing if path 
doesn't exist
+  val status: Option[FileStatus] = try Option(fs.getFileStatus(path)) 
catch {
+case _: FileNotFoundException =>
+  logWarning(s"The directory $path was not found. Was it deleted 
very recently?")
+  None
+  }
 
-childStatuses.map {
-  case f: LocatedFileStatus => f
-
-  // NOTE:
-  //
-  // - Although S3/S3A/S3N file system can be quite slow for 
remote file metadata
-  //   operations, calling `getFileBlockLocations` does no harm 
here since these file system
-  //   implementations don't actually issue RPC for this method.
-  //
-  // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should not
-  //   be a big deal since we always use to 
`listLeafFilesInParallel` when the number of
-  //   paths exceeds threshold.
-  case f =>
-if (f.isDirectory 

[GitHub] spark pull request #15235: [SPARK-17661][SQL] Consolidate various listLeafFi...

2016-09-26 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/15235#discussion_r80574449
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 ---
@@ -82,73 +85,185 @@ class ListingFileCatalog(
* This is publicly visible for testing.
*/
   def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = 
{
-if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
-  HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
-} else {
-  // Right now, the number of paths is less than the value of
-  // parallelPartitionDiscoveryThreshold. So, we will list file 
statues at the driver.
-  // If there is any child that has more files than the threshold, we 
will use parallel
-  // listing.
-
-  // Dummy jobconf to get to the pathFilter defined in configuration
-  val jobConf = new JobConf(hadoopConf, this.getClass)
-  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
-
-  val statuses: Seq[FileStatus] = paths.flatMap { path =>
-val fs = path.getFileSystem(hadoopConf)
-logTrace(s"Listing $path on driver")
-
-val childStatuses = {
-  try {
-val stats = fs.listStatus(path)
-if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-  } catch {
-case _: FileNotFoundException =>
-  logWarning(s"The directory $path was not found. Was it 
deleted very recently?")
-  Array.empty[FileStatus]
-  }
-}
+val files =
+  if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
+ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
+  } else {
+ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf)
+  }
+
+mutable.LinkedHashSet(files: _*)
+  }
+
+  override def equals(other: Any): Boolean = other match {
+case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet
+case _ => false
+  }
+
+  override def hashCode(): Int = paths.toSet.hashCode()
+}
+
+
+object ListingFileCatalog extends Logging {
+
+  // `FileStatus` is Writable but not serializable.  What make it worse, 
somehow it doesn't play
+  // well with `SerializableWritable`.  So there seems to be no way to 
serialize a `FileStatus`.
+  // Here we use `SerializableFileStatus` to extract key components of a 
`FileStatus` to serialize
+  // it from executor side and reconstruct it on driver side.
+  private case class SerializableBlockLocation(
+  names: Array[String],
+  hosts: Array[String],
+  offset: Long,
+  length: Long)
+
+  private case class SerializableFileStatus(
+  path: String,
+  length: Long,
+  isDir: Boolean,
+  blockReplication: Short,
+  blockSize: Long,
+  modificationTime: Long,
+  accessTime: Long,
+  blockLocations: Array[SerializableBlockLocation])
+
+  /**
+   * List a collection of path recursively.
+   */
+  private def listLeafFilesInSerial(
+  paths: Seq[Path],
+  hadoopConf: Configuration): Seq[FileStatus] = {
+// Dummy jobconf to get to the pathFilter defined in configuration
+val jobConf = new JobConf(hadoopConf, this.getClass)
+val filter = FileInputFormat.getInputPathFilter(jobConf)
+
+paths.flatMap { path =>
+  logTrace(s"Listing $path")
+  val fs = path.getFileSystem(hadoopConf)
+
+  // [SPARK-17599] Prevent ListingFileCatalog from failing if path 
doesn't exist
+  val status: Option[FileStatus] = try Option(fs.getFileStatus(path)) 
catch {
--- End diff --

I don't think you need this. You are increasing the number of `getStatus` 
calls that we need to make. There is no guarantee that the folder will exist 
once `listLeafFiles0` is called.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15235: [SPARK-17661][SQL] Consolidate various listLeafFi...

2016-09-26 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/15235#discussion_r80581975
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 ---
@@ -82,73 +85,185 @@ class ListingFileCatalog(
* This is publicly visible for testing.
*/
   def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = 
{
-if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
-  HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
-} else {
-  // Right now, the number of paths is less than the value of
-  // parallelPartitionDiscoveryThreshold. So, we will list file 
statues at the driver.
-  // If there is any child that has more files than the threshold, we 
will use parallel
-  // listing.
-
-  // Dummy jobconf to get to the pathFilter defined in configuration
-  val jobConf = new JobConf(hadoopConf, this.getClass)
-  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
-
-  val statuses: Seq[FileStatus] = paths.flatMap { path =>
-val fs = path.getFileSystem(hadoopConf)
-logTrace(s"Listing $path on driver")
-
-val childStatuses = {
-  try {
-val stats = fs.listStatus(path)
-if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-  } catch {
-case _: FileNotFoundException =>
-  logWarning(s"The directory $path was not found. Was it 
deleted very recently?")
-  Array.empty[FileStatus]
-  }
-}
+val files =
+  if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
+ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
+  } else {
+ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf)
+  }
+
+mutable.LinkedHashSet(files: _*)
+  }
+
+  override def equals(other: Any): Boolean = other match {
+case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet
+case _ => false
+  }
+
+  override def hashCode(): Int = paths.toSet.hashCode()
+}
+
+
+object ListingFileCatalog extends Logging {
+
+  // `FileStatus` is Writable but not serializable.  What make it worse, 
somehow it doesn't play
+  // well with `SerializableWritable`.  So there seems to be no way to 
serialize a `FileStatus`.
+  // Here we use `SerializableFileStatus` to extract key components of a 
`FileStatus` to serialize
+  // it from executor side and reconstruct it on driver side.
+  private case class SerializableBlockLocation(
+  names: Array[String],
+  hosts: Array[String],
+  offset: Long,
+  length: Long)
+
+  private case class SerializableFileStatus(
+  path: String,
+  length: Long,
+  isDir: Boolean,
+  blockReplication: Short,
+  blockSize: Long,
+  modificationTime: Long,
+  accessTime: Long,
+  blockLocations: Array[SerializableBlockLocation])
+
+  /**
+   * List a collection of path recursively.
+   */
+  private def listLeafFilesInSerial(
+  paths: Seq[Path],
+  hadoopConf: Configuration): Seq[FileStatus] = {
+// Dummy jobconf to get to the pathFilter defined in configuration
+val jobConf = new JobConf(hadoopConf, this.getClass)
+val filter = FileInputFormat.getInputPathFilter(jobConf)
+
+paths.flatMap { path =>
+  logTrace(s"Listing $path")
+  val fs = path.getFileSystem(hadoopConf)
+
+  // [SPARK-17599] Prevent ListingFileCatalog from failing if path 
doesn't exist
+  val status: Option[FileStatus] = try Option(fs.getFileStatus(path)) 
catch {
+case _: FileNotFoundException =>
+  logWarning(s"The directory $path was not found. Was it deleted 
very recently?")
+  None
+  }
 
-childStatuses.map {
-  case f: LocatedFileStatus => f
-
-  // NOTE:
-  //
-  // - Although S3/S3A/S3N file system can be quite slow for 
remote file metadata
-  //   operations, calling `getFileBlockLocations` does no harm 
here since these file system
-  //   implementations don't actually issue RPC for this method.
-  //
-  // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should not
-  //   be a big deal since we always use to 
`listLeafFilesInParallel` when the number of
-  //   paths exceeds threshold.
-  case f =>
-if (f.isDirectory 

[GitHub] spark pull request #15235: [SPARK-17661][SQL] Consolidate various listLeafFi...

2016-09-26 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/15235#discussion_r80582465
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 ---
@@ -82,73 +85,185 @@ class ListingFileCatalog(
* This is publicly visible for testing.
*/
   def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = 
{
-if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
-  HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
-} else {
-  // Right now, the number of paths is less than the value of
-  // parallelPartitionDiscoveryThreshold. So, we will list file 
statues at the driver.
-  // If there is any child that has more files than the threshold, we 
will use parallel
-  // listing.
-
-  // Dummy jobconf to get to the pathFilter defined in configuration
-  val jobConf = new JobConf(hadoopConf, this.getClass)
-  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
-
-  val statuses: Seq[FileStatus] = paths.flatMap { path =>
-val fs = path.getFileSystem(hadoopConf)
-logTrace(s"Listing $path on driver")
-
-val childStatuses = {
-  try {
-val stats = fs.listStatus(path)
-if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-  } catch {
-case _: FileNotFoundException =>
-  logWarning(s"The directory $path was not found. Was it 
deleted very recently?")
-  Array.empty[FileStatus]
-  }
-}
+val files =
+  if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
+ListingFileCatalog.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession)
+  } else {
+ListingFileCatalog.listLeafFilesInSerial(paths, hadoopConf)
+  }
+
+mutable.LinkedHashSet(files: _*)
+  }
+
+  override def equals(other: Any): Boolean = other match {
+case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet
+case _ => false
+  }
+
+  override def hashCode(): Int = paths.toSet.hashCode()
+}
+
+
+object ListingFileCatalog extends Logging {
+
+  // `FileStatus` is Writable but not serializable.  What make it worse, 
somehow it doesn't play
+  // well with `SerializableWritable`.  So there seems to be no way to 
serialize a `FileStatus`.
+  // Here we use `SerializableFileStatus` to extract key components of a 
`FileStatus` to serialize
+  // it from executor side and reconstruct it on driver side.
+  private case class SerializableBlockLocation(
+  names: Array[String],
+  hosts: Array[String],
+  offset: Long,
+  length: Long)
+
+  private case class SerializableFileStatus(
+  path: String,
+  length: Long,
+  isDir: Boolean,
+  blockReplication: Short,
+  blockSize: Long,
+  modificationTime: Long,
+  accessTime: Long,
+  blockLocations: Array[SerializableBlockLocation])
+
+  /**
+   * List a collection of path recursively.
+   */
+  private def listLeafFilesInSerial(
+  paths: Seq[Path],
+  hadoopConf: Configuration): Seq[FileStatus] = {
+// Dummy jobconf to get to the pathFilter defined in configuration
+val jobConf = new JobConf(hadoopConf, this.getClass)
+val filter = FileInputFormat.getInputPathFilter(jobConf)
+
+paths.flatMap { path =>
+  logTrace(s"Listing $path")
+  val fs = path.getFileSystem(hadoopConf)
+
+  // [SPARK-17599] Prevent ListingFileCatalog from failing if path 
doesn't exist
+  val status: Option[FileStatus] = try Option(fs.getFileStatus(path)) 
catch {
+case _: FileNotFoundException =>
+  logWarning(s"The directory $path was not found. Was it deleted 
very recently?")
+  None
+  }
 
-childStatuses.map {
-  case f: LocatedFileStatus => f
-
-  // NOTE:
-  //
-  // - Although S3/S3A/S3N file system can be quite slow for 
remote file metadata
-  //   operations, calling `getFileBlockLocations` does no harm 
here since these file system
-  //   implementations don't actually issue RPC for this method.
-  //
-  // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should not
-  //   be a big deal since we always use to 
`listLeafFilesInParallel` when the number of
-  //   paths exceeds threshold.
-  case f =>
-if (f.isDirectory 

[GitHub] spark issue #13680: [SPARK-15962][SQL] Introduce implementation with a dense...

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

https://github.com/apache/spark/pull/13680
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65930/
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 #13680: [SPARK-15962][SQL] Introduce implementation with a dense...

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

https://github.com/apache/spark/pull/13680
  
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 #13680: [SPARK-15962][SQL] Introduce implementation with a dense...

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

https://github.com/apache/spark/pull/13680
  
**[Test build #65930 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65930/consoleFull)**
 for PR 13680 at commit 
[`2ef6e3b`](https://github.com/apache/spark/commit/2ef6e3bdbd16c7f7b9ff006d48382e108ed37eef).
 * 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 #14659: [SPARK-16757] Set up Spark caller context to HDFS and YA...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14659: [SPARK-16757] Set up Spark caller context to HDFS...

2016-09-26 Thread Sherry302
Github user Sherry302 commented on a diff in the pull request:

https://github.com/apache/spark/pull/14659#discussion_r80579059
  
--- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala ---
@@ -2421,6 +2421,69 @@ private[spark] object Utils extends Logging {
 }
 
 /**
+ * An utility class used to set up Spark caller contexts to HDFS and Yarn. 
The `context` will be
+ * constructed by parameters passed in.
+ * When Spark applications run on Yarn and HDFS, its caller contexts will 
be written into Yarn RM
+ * audit log and hdfs-audit.log. That can help users to better diagnose 
and understand how
+ * specific applications impacting parts of the Hadoop system and 
potential problems they may be
+ * creating (e.g. overloading NN). As HDFS mentioned in HDFS-9184, for a 
given HDFS operation, it's
+ * very helpful to track which upper level job issues it.
+ *
+ * @param from who sets up the caller context (TASK, CLIENT, APPMASTER)
+ *
+ * The parameters below are optional:
+ * @param appId id of the app this task belongs to
+ * @param appAttemptId attempt id of the app this task belongs to
+ * @param jobId id of the job this task belongs to
+ * @param stageId id of the stage this task belongs to
+ * @param stageAttemptId attempt id of the stage this task belongs to
+ * @param taskId task id
+ * @param taskAttemptNumber task attempt id
+ * @since 2.0.1
+ */
+private[spark] class CallerContext(
+   from: String,
+   appId: Option[String] = None,
+   appAttemptId: Option[String] = None,
+   jobId: Option[Int] = None,
+   stageId: Option[Int] = None,
+   stageAttemptId: Option[Int] = None,
+   taskId: Option[Long] = None,
+   taskAttemptNumber: Option[Int] = None) extends Logging {
+
+   val AppId = if (appId.isDefined) s"_${appId.get}" else ""
--- End diff --

Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14659: [SPARK-16757] Set up Spark caller context to HDFS and YA...

2016-09-26 Thread Sherry302
Github user Sherry302 commented on the issue:

https://github.com/apache/spark/pull/14659
  
Hi, @tgravescs Thanks a lot for the comments. I have updated the PR to 
rename local vals and remove the `@since` in `Utils.scala`.


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

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



[GitHub] spark issue #14897: [SPARK-17338][SQL] add global temp view

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

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


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

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



[GitHub] spark issue #14897: [SPARK-17338][SQL] add global temp view

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

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


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

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



[GitHub] spark pull request #14659: [SPARK-16757] Set up Spark caller context to HDFS...

2016-09-26 Thread Sherry302
Github user Sherry302 commented on a diff in the pull request:

https://github.com/apache/spark/pull/14659#discussion_r80579032
  
--- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala ---
@@ -2421,6 +2421,69 @@ private[spark] object Utils extends Logging {
 }
 
 /**
+ * An utility class used to set up Spark caller contexts to HDFS and Yarn. 
The `context` will be
+ * constructed by parameters passed in.
+ * When Spark applications run on Yarn and HDFS, its caller contexts will 
be written into Yarn RM
+ * audit log and hdfs-audit.log. That can help users to better diagnose 
and understand how
+ * specific applications impacting parts of the Hadoop system and 
potential problems they may be
+ * creating (e.g. overloading NN). As HDFS mentioned in HDFS-9184, for a 
given HDFS operation, it's
+ * very helpful to track which upper level job issues it.
+ *
+ * @param from who sets up the caller context (TASK, CLIENT, APPMASTER)
+ *
+ * The parameters below are optional:
+ * @param appId id of the app this task belongs to
+ * @param appAttemptId attempt id of the app this task belongs to
+ * @param jobId id of the job this task belongs to
+ * @param stageId id of the stage this task belongs to
+ * @param stageAttemptId attempt id of the stage this task belongs to
+ * @param taskId task id
+ * @param taskAttemptNumber task attempt id
+ * @since 2.0.1
--- End diff --

Done.


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

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



[GitHub] spark issue #14897: [SPARK-17338][SQL] add global temp view

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

https://github.com/apache/spark/pull/14897
  
**[Test build #65929 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65929/consoleFull)**
 for PR 14897 at commit 
[`98d201b`](https://github.com/apache/spark/commit/98d201b71c5899d59ee3a63f0f153e70f9545308).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class SparkContext(config: SparkConf) extends Logging `
  * `// ('scaled' = +Infinity). However in the case that this class 
also has`
  * `// 0 probability, the class will not be selected ('scaled' is 
NaN).`
  * `  final val thresholds: DoubleArrayParam = new DoubleArrayParam(this, 
\"thresholds\", \"Thresholds in multi-class classification to adjust the 
probability of predicting each class. Array must have length equal to the 
number of classes, with values > 0 excepting that at most one value may be 0. 
The class with largest value p/t is predicted, where p is the original 
probability of that class and t is the class's threshold\", (t: Array[Double]) 
=> t.forall(_ >= 0) && t.count(_ == 0) <= 1)`
  * `thresholds = Param(Params._dummy(), \"thresholds\", \"Thresholds 
in multi-class classification to adjust the probability of predicting each 
class. Array must have length equal to the number of classes, with values > 0, 
excepting that at most one value may be 0. The class with largest value p/t is 
predicted, where p is the original probability of that class and t is the 
class's threshold.\", typeConverter=TypeConverters.toListFloat)`
  * `case class SortOrder(child: Expression, direction: SortDirection, 
nullOrdering: NullOrdering)`
  * `  case class FileEntry(path: String, timestamp: Timestamp, batchId: 
Long) extends Serializable`
  * `trait Offset 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 #15246: [MINOR][SQL] Use resource path for test_script.sh

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15246: [MINOR][SQL] Use resource path for test_script.sh

2016-09-26 Thread Sherry302
GitHub user Sherry302 opened a pull request:

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

[MINOR][SQL] Use resource path for test_script.sh

## What changes were proposed in this pull request?
This PR modified the test case `test("script")` to use resource path for 
`test_script.sh`. Make the test case portable (even in IntelliJ).


## How was this patch tested?
Passed the test case.
Before:
Run `test("script")` in IntelliJ:
```
Caused by: org.apache.spark.SparkException: Subprocess exited with status 
127. Error: bash: src/test/resources/test_script.sh: No such file or directory 
```
After:
Test passed.


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

$ git pull https://github.com/Sherry302/spark hivetest

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

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


commit d799eea4ca3e3ad0fc71fe49985e6bc51f197158
Author: Weiqing Yang 
Date:   2016-09-26T21:15:06Z

[MINOR][SQL] Use resource path for test_script.sh




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14547: [SPARK-16718][MLlib] gbm-style treeboost

2016-09-26 Thread jkbradley
Github user jkbradley commented on the issue:

https://github.com/apache/spark/pull/14547
  
@sethah AFAIK, the original gradient boosting algorithm was generic, not 
specific to trees.  That's Algorithm 1 from 
[https://statweb.stanford.edu/~jhf/ftp/trebst.pdf] and is what MLlib has 
currently.

I agree with your intuition about options 3 > 2 > 1 and encouraging users 
to use option 3 via our API.  I'd be OK with disallowing option 1.  As a 
software engineer, I'd want to allow 1 for backwards API compatibility, where 
behavior and algorithms are part of the API.  But as an ML person, I'd be Ok 
with not even allowing 1 in the future to prevent users from doing the wrong 
thing.  Combining these, I'd recommend:
* For now, we make 2 the default behavior but still allow 1.  (as in this 
PR)
* In the future, we make 3 the default behavior, maybe allow 2, and do not 
allow 1.

> "loss-based"  What exactly does that mean to the user?

If this is unclear, then let's make the documentation for that Param 
clearer and/or use a more intuitive name such as "auto."



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

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



[GitHub] spark issue #14818: [SPARK-17157][SPARKR][WIP]: Add multiclass logistic regr...

2016-09-26 Thread wangmiao1981
Github user wangmiao1981 commented on the issue:

https://github.com/apache/spark/pull/14818
  
For example:
`LogisticRegression` has `Threshold`, `Thresholds` and `AggregationDepth` 
etc as parameters.

`GeneralizedLinearRegression` doesn't have such parameters. So the 
interface 
function(data, formula, family = gaussian, tol = 1e-6, maxIter = 25, 
weightCol = NULL,
   regParam = 0.0)

should be modified to accept these parameters.


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

2016-09-26 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15245#discussion_r80568308
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala
 ---
@@ -30,7 +31,8 @@ import 
org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
  * An adaptor from a [[PartitionedFile]] to an [[Iterator]] of [[Text]], 
which are all of the lines
  * in that file.
  */
-class HadoopFileLinesReader(file: PartitionedFile, conf: Configuration) 
extends Iterator[Text] {
+class HadoopFileLinesReader(
+file: PartitionedFile, conf: Configuration) extends Iterator[Text] 
with Closeable {
   private val iterator = {
--- End diff --

This is a `RecordReaderIterator`, whose memory footprint should become 
practically nothing after `close()` is called, hence my decision to not null 
things out here.


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

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



[GitHub] spark issue #13998: [SPARK-12177][Streaming][Kafka] limit api surface area

2016-09-26 Thread tdas
Github user tdas commented on the issue:

https://github.com/apache/spark/pull/13998
  
@koeninger Could you take a look at this test flakiness in

https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/1792/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #9162: [SPARK-10655][SQL] Adding additional data type mappings t...

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

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


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

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

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


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

2016-09-26 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15245#discussion_r80568112
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/RecordReaderIterator.scala
 ---
@@ -27,7 +29,8 @@ import org.apache.spark.sql.catalyst.InternalRow
  * Note that this returns [[Object]]s instead of [[InternalRow]] because 
we rely on erasure to pass
  * column batches by pretending they are rows.
  */
-class RecordReaderIterator[T](rowReader: RecordReader[_, T]) extends 
Iterator[T] {
+class RecordReaderIterator[T](
+private[this] var rowReader: RecordReader[_, T]) extends Iterator[T] 
with Closeable {
--- End diff --

By nulling out the `rowReader` I think that this will prevent memory 
consumption from becoming too high in the list of task completion callbacks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15155: [SPARK-17477][SQL] SparkSQL cannot handle schema ...

2016-09-26 Thread wgtmac
Github user wgtmac closed the pull request at:

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


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

2016-09-26 Thread srowen
Github user srowen commented on the issue:

https://github.com/apache/spark/pull/15245
  
Nice one, looks like something that can only help.


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

2016-09-26 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15245#discussion_r80567995
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/RecordReaderIterator.scala
 ---
@@ -52,4 +55,18 @@ class RecordReaderIterator[T](rowReader: RecordReader[_, 
T]) extends Iterator[T]
 havePair = false
 rowReader.getCurrentValue
   }
+
+  override def close(): Unit = {
+if (rowReader != null) {
+  try {
+// Close the reader and release it. Note: it's very important that 
we don't close the
--- End diff --

This comment is copied from `NewHadoopRdd`, which contains similar 
defensive programming.


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

2016-09-26 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15245#discussion_r80567861
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/RecordReaderIterator.scala
 ---
@@ -38,7 +41,7 @@ class RecordReaderIterator[T](rowReader: RecordReader[_, 
T]) extends Iterator[T]
 // Close and release the reader here; close() will also be called 
when the task
 // completes, but for tasks that read from many files, it helps to 
release the
 // resources early.
-rowReader.close()
--- End diff --

This is `CompletionIterator`-style cleanup.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14426: [SPARK-16475][SQL] Broadcast Hint for SQL Queries

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

https://github.com/apache/spark/pull/14426
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65927/
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 #14959: [SPARK-17387][PYSPARK] Creating SparkContext() from pyth...

2016-09-26 Thread BryanCutler
Github user BryanCutler commented on the issue:

https://github.com/apache/spark/pull/14959
  
I added a few comments @zjffdu.  I also tested this PR out and looks good


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14426: [SPARK-16475][SQL] Broadcast Hint for SQL Queries

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

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


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

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



[GitHub] spark pull request #15244: Update spark-standalone.md to fix link

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #13680: [SPARK-15962][SQL] Introduce implementation with a dense...

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

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


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

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



[GitHub] spark pull request #15245: [SPARK-17666] Ensure that RecordReaders are close...

2016-09-26 Thread JoshRosen
GitHub user JoshRosen opened a pull request:

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

[SPARK-17666] Ensure that RecordReaders are closed by data source file scans

## What changes were proposed in this pull request?

This patch addresses a potential cause of resource leaks in data source 
file scans. As reported in 
[SPARK-17666](https://issues.apache.org/jira/browse/SPARK-17666), tasks which 
do not fully-consume their input may cause file handles / network connections 
(e.g. S3 connections) to be leaked. Spark's `NewHadoopRDD` uses a TaskContext 
callback to [close its record 
readers](https://github.com/apache/spark/blame/master/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L208),
 but the new data source file scans will only close record readers once their 
iterators are fully-consumed.

This patch modifies `RecordReaderIterator` and `HadoopFileLinesReader` to 
add `close()` methods and modifies all six implementations of 
`FileFormat.buildReader()` to register TaskContext task completion callbacks to 
guarantee that cleanup is eventually performed.

## How was this patch tested?

Tested manually for now.

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

$ git pull https://github.com/JoshRosen/spark SPARK-17666-close-recordreader

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

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


commit d804e025c2b4a8799f38f2f67beba1d12e224180
Author: Josh Rosen 
Date:   2016-09-26T20:28:12Z

Add close() to RecordReaderIterator and HadoopLinesReader

commit e4b8577ed71a30f4ad220cd1a2f19a8edd596c64
Author: Josh Rosen 
Date:   2016-09-26T20:29:24Z

Register close() callbacks in all implementations of 
FileFormat.buildReader()




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #13680: [SPARK-15962][SQL] Introduce implementation with a dense...

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

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


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

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



[GitHub] spark issue #14426: [SPARK-16475][SQL] Broadcast Hint for SQL Queries

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

https://github.com/apache/spark/pull/14426
  
**[Test build #65927 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65927/consoleFull)**
 for PR 14426 at commit 
[`e9ba01e`](https://github.com/apache/spark/commit/e9ba01efc725f6a99025f90d2ac29a79588dc6f9).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class Hint(name: String, parameters: Seq[String], child: 
LogicalPlan) extends UnaryNode `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #13680: [SPARK-15962][SQL] Introduce implementation with a dense...

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

https://github.com/apache/spark/pull/13680
  
**[Test build #65920 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65920/consoleFull)**
 for PR 13680 at commit 
[`2ef6e3b`](https://github.com/apache/spark/commit/2ef6e3bdbd16c7f7b9ff006d48382e108ed37eef).
 * This patch **fails from timeout after a configured wait of \`250m\`**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark pull request #14959: [SPARK-17387][PYSPARK] Creating SparkContext() fr...

2016-09-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/14959#discussion_r80567211
  
--- Diff: python/pyspark/conf.py ---
@@ -118,28 +130,28 @@ def setIfMissing(self, key, value):
 
 def setMaster(self, value):
 """Set master URL to connect to."""
-self._jconf.setMaster(value)
+self.set("spark.master", value)
 return self
 
 def setAppName(self, value):
 """Set application name."""
-self._jconf.setAppName(value)
+self.set("spark.app.name", value)
 return self
 
 def setSparkHome(self, value):
 """Set path where Spark is installed on worker nodes."""
-self._jconf.setSparkHome(value)
+self.set("spark.home", value)
 return self
 
 def setExecutorEnv(self, key=None, value=None, pairs=None):
 """Set an environment variable to be passed to executors."""
 if (key is not None and pairs is not None) or (key is None and 
pairs is None):
 raise Exception("Either pass one key-value pair or a list of 
pairs")
 elif key is not None:
-self._jconf.setExecutorEnv(key, value)
+self.set("spark.executorEnv." + key, value)
--- End diff --

since you duplicated the property prefix "spark.executorEnv." from the 
Scala file, it might be good to make a unit test that ensures calling 
`setExecutorEnv` in both Python and Scala/Java actually sets a property with 
the same prefix.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15244: Update spark-standalone.md to fix link

2016-09-26 Thread andrewor14
Github user andrewor14 commented on the issue:

https://github.com/apache/spark/pull/15244
  
Thanks merged into master 2.0 and 1.6


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14959: [SPARK-17387][PYSPARK] Creating SparkContext() fr...

2016-09-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/14959#discussion_r80566426
  
--- Diff: python/pyspark/java_gateway.py ---
@@ -41,7 +41,7 @@ def can_convert_list(self, obj):
 ListConverter.can_convert = can_convert_list
 
 
-def launch_gateway():
+def launch_gateway(conf=None):
--- End diff --

It might be helpful to have a docstring with param description of `conf` 
here


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

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



[GitHub] spark issue #15102: [SPARK-17346][SQL] Add Kafka source for Structured Strea...

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

https://github.com/apache/spark/pull/15102
  
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 #15102: [SPARK-17346][SQL] Add Kafka source for Structured Strea...

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

https://github.com/apache/spark/pull/15102
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65926/
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 #15102: [SPARK-17346][SQL] Add Kafka source for Structured Strea...

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

https://github.com/apache/spark/pull/15102
  
**[Test build #65926 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65926/consoleFull)**
 for PR 15102 at commit 
[`755ceaa`](https://github.com/apache/spark/commit/755ceaa3531a690f755982a55c71f977c3039bc0).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `// ('scaled' = +Infinity). However in the case that this class 
also has`
  * `// 0 probability, the class will not be selected ('scaled' is 
NaN).`
  * `  final val thresholds: DoubleArrayParam = new DoubleArrayParam(this, 
\"thresholds\", \"Thresholds in multi-class classification to adjust the 
probability of predicting each class. Array must have length equal to the 
number of classes, with values > 0 excepting that at most one value may be 0. 
The class with largest value p/t is predicted, where p is the original 
probability of that class and t is the class's threshold\", (t: Array[Double]) 
=> t.forall(_ >= 0) && t.count(_ == 0) <= 1)`
  * `thresholds = Param(Params._dummy(), \"thresholds\", \"Thresholds 
in multi-class classification to adjust the probability of predicting each 
class. Array must have length equal to the number of classes, with values > 0, 
excepting that at most one value may be 0. The class with largest value p/t is 
predicted, where p is the original probability of that class and t is the 
class's threshold.\", typeConverter=TypeConverters.toListFloat)`
  * `case class SortOrder(child: Expression, direction: SortDirection, 
nullOrdering: NullOrdering)`


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

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



[GitHub] spark pull request #11119: [SPARK-10780][ML] Add an initial model to kmeans

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

https://github.com/apache/spark/pull/9#discussion_r80565861
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala ---
@@ -107,24 +133,34 @@ trait DefaultReadWriteTest extends TempDirectory { 
self: Suite =>
 }
 val model = estimator.fit(dataset)
 
-// Test Estimator save/load
-val estimator2 = testDefaultReadWrite(estimator)
-testParams.foreach { case (p, v) =>
-  val param = estimator.getParam(p)
-  assert(estimator.get(param).get === estimator2.get(param).get)
+val testFunctions = if (testParams.contains("initialModel")) {
+  Map(("initialModel", checkModelData.asInstanceOf[(Any, Any) => 
Unit]))
--- End diff --

On second thought, it really might actually be easiest to override the 
equals method. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14959: [SPARK-17387][PYSPARK] Creating SparkContext() fr...

2016-09-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/14959#discussion_r80565092
  
--- Diff: python/pyspark/conf.py ---
@@ -149,35 +161,53 @@ def setAll(self, pairs):
 :param pairs: list of key-value pairs to set
 """
 for (k, v) in pairs:
-self._jconf.set(k, v)
+self.set(k, v)
 return self
 
 def get(self, key, defaultValue=None):
 """Get the configured value for some key, or return a default 
otherwise."""
 if defaultValue is None:   # Py4J doesn't call the right get() if 
we pass None
-if not self._jconf.contains(key):
-return None
-return self._jconf.get(key)
+if self._jconf:
+if not self._jconf.contains(key):
+return None
+return self._jconf.get(key)
+else:
+if key not in self._conf:
+return None
+return self._conf[key]
 else:
-return self._jconf.get(key, defaultValue)
+if self._jconf:
+return self._jconf.get(key, defaultValue)
+else:
+return self._conf.get(key, defaultValue)
 
 def getAll(self):
 """Get all values as a list of key-value pairs."""
 pairs = []
-for elem in self._jconf.getAll():
-pairs.append((elem._1(), elem._2()))
+if self._jconf:
+for elem in self._jconf.getAll():
+pairs.append((elem._1(), elem._2()))
+else:
+for k, v in self._conf.items():
+pairs.append((k, v))
 return pairs
 
 def contains(self, key):
 """Does this configuration contain a given key?"""
-return self._jconf.contains(key)
+if self._jconf:
+return self._jconf.contains(key)
+else:
+return key in self._conf
 
 def toDebugString(self):
 """
 Returns a printable version of the configuration, as a list of
 key=value pairs, one per line.
 """
-return self._jconf.toDebugString()
+if self._jconf:
+return self._jconf.toDebugString()
+else:
+return '\n'.join('%s=%s' % (k, v) for k, v in 
self._conf.items())
--- End diff --

maybe add a unit test to make sure the 2 ways to make a debug string are 
the same?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14959: [SPARK-17387][PYSPARK] Creating SparkContext() fr...

2016-09-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/14959#discussion_r80564836
  
--- Diff: python/pyspark/conf.py ---
@@ -149,35 +161,53 @@ def setAll(self, pairs):
 :param pairs: list of key-value pairs to set
 """
 for (k, v) in pairs:
-self._jconf.set(k, v)
+self.set(k, v)
 return self
 
 def get(self, key, defaultValue=None):
 """Get the configured value for some key, or return a default 
otherwise."""
 if defaultValue is None:   # Py4J doesn't call the right get() if 
we pass None
-if not self._jconf.contains(key):
-return None
-return self._jconf.get(key)
+if self._jconf:
+if not self._jconf.contains(key):
+return None
+return self._jconf.get(key)
+else:
+if key not in self._conf:
+return None
+return self._conf[key]
 else:
-return self._jconf.get(key, defaultValue)
+if self._jconf:
+return self._jconf.get(key, defaultValue)
+else:
+return self._conf.get(key, defaultValue)
 
 def getAll(self):
 """Get all values as a list of key-value pairs."""
 pairs = []
-for elem in self._jconf.getAll():
-pairs.append((elem._1(), elem._2()))
+if self._jconf:
+for elem in self._jconf.getAll():
+pairs.append((elem._1(), elem._2()))
+else:
+for k, v in self._conf.items():
+pairs.append((k, v))
--- End diff --

This could be simplified:

```python
if self._jconf is not None:
return [(elem._1(), elem._2()) for elem in self._jconf.getAll()]
else:
return self._conf.items()


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15225: [SPARK-17652] Fix confusing exception message whi...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15225: [SPARK-17652] Fix confusing exception message while rese...

2016-09-26 Thread yhuai
Github user yhuai commented on the issue:

https://github.com/apache/spark/pull/15225
  
LGTM. Merging to master and branch 2.0.


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

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



[GitHub] spark pull request #14959: [SPARK-17387][PYSPARK] Creating SparkContext() fr...

2016-09-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/14959#discussion_r80561545
  
--- Diff: python/pyspark/conf.py ---
@@ -101,13 +101,25 @@ def __init__(self, loadDefaults=True, _jvm=None, 
_jconf=None):
 self._jconf = _jconf
 else:
 from pyspark.context import SparkContext
-SparkContext._ensure_initialized()
 _jvm = _jvm or SparkContext._jvm
-self._jconf = _jvm.SparkConf(loadDefaults)
+
+if _jvm:
+# JVM is created, so create self._jconf directly through 
JVM
+self._jconf = _jvm.SparkConf(loadDefaults)
+self._conf = None
+else:
+# JVM is not created, so store data in self._conf first
+self._jconf = None
+self._conf = {}
 
 def set(self, key, value):
 """Set a configuration property."""
-self._jconf.set(key, unicode(value))
+# Try to set self._jconf first if JVM is created, set self._conf 
if JVM is not created yet.
+if self._jconf:
--- End diff --

minor:  I think it's generally better and would be more consistent with the 
rest of the code if you made these checks against a `None` value, here and 
other places in this PR.  For example:
```python
if self._jconf is not None:
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14803: [SPARK-17153][SQL] Should read partition data whe...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14959: [SPARK-17387][PYSPARK] Creating SparkContext() fr...

2016-09-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/14959#discussion_r80561001
  
--- Diff: python/pyspark/conf.py ---
@@ -101,13 +101,25 @@ def __init__(self, loadDefaults=True, _jvm=None, 
_jconf=None):
 self._jconf = _jconf
 else:
 from pyspark.context import SparkContext
-SparkContext._ensure_initialized()
 _jvm = _jvm or SparkContext._jvm
-self._jconf = _jvm.SparkConf(loadDefaults)
+
+if _jvm:
+# JVM is created, so create self._jconf directly through 
JVM
+self._jconf = _jvm.SparkConf(loadDefaults)
+self._conf = None
+else:
+# JVM is not created, so store data in self._conf first
+self._jconf = None
+self._conf = {}
 
 def set(self, key, value):
 """Set a configuration property."""
-self._jconf.set(key, unicode(value))
+# Try to set self._jconf first if JVM is created, set self._conf 
if JVM is not created yet.
+if self._jconf:
+self._jconf.set(key, unicode(value))
+else:
+# Don't use unicode for self._conf, otherwise we will get 
exception when launching jvm.
+self._conf[key] = value
--- End diff --

Could you just cast the value to unicode here also?  Then it would be 
consistent with the Java class and you wouldn't need to change the doctest 
above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15148: [SPARK-5992][ML] Locality Sensitive Hashing

2016-09-26 Thread jkbradley
Github user jkbradley commented on the issue:

https://github.com/apache/spark/pull/15148
  
* Do we want to use the subpackage ```spark.ml.feature.lsh``` or just put 
the classes under ```spark.ml.feature```?  This would be the first division of 
```feature```.  I'd prefer not using subpackage ```lsh``` to be consistent.

> (MLnick)  I can see for binary (i.e. hamming dist) that Array[Boolean] is 
attractive as a kind of type safety thing, but still I think a Vector interface 
is more natural.

We could allow both, though that would require changing the LSH 
abstraction.  In the future, I do hope ML algorithms become more relaxed in 
terms of which Catalyst types they accept.

> (MLnick) efficiently support top-k recommendations across an entire 
dataset

I like the idea of returning results with the top-k values since I agree 
it's closer to what most users would want, versus specifying a threshold.  I 
assume it would be more expensive, requiring some grouping of the data.  
Perhaps we can add it in a follow-up PR.

@Yunni Thanks for sending the PR! I'd be happy to make a more detailed 
review pass, though I'll wait for some of the comments to be addressed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14803: [SPARK-17153][SQL] Should read partition data when readi...

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

https://github.com/apache/spark/pull/14803
  
Thanks, I'm going to merge this to master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15244: Update spark-standalone.md to fix link

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

https://github.com/apache/spark/pull/15244
  
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 #15148: [SPARK-5992][ML] Locality Sensitive Hashing

2016-09-26 Thread jkbradley
Github user jkbradley commented on a diff in the pull request:

https://github.com/apache/spark/pull/15148#discussion_r80559677
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/lsh/LSH.scala ---
@@ -0,0 +1,304 @@
+/*
+ * 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.ml.feature.lsh
+
+import scala.util.Random
+
+import org.apache.spark.ml.{Estimator, Model}
+import org.apache.spark.ml.linalg.{Vector, VectorUDT}
+import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
+import org.apache.spark.sql._
+import org.apache.spark.sql.expressions.UserDefinedFunction
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.types._
+
+/**
+ * Params for [[LSH]].
+ */
+private[ml] trait LSHParams extends HasInputCol with HasOutputCol {
+  /**
+   * Param for output dimension.
+   *
+   * @group param
+   */
+  final val outputDim: IntParam = new IntParam(this, "outputDim", "output 
dimension",
+ParamValidators.gt(0))
+
+  /** @group getParam */
+  final def getOutputDim: Int = $(outputDim)
+
+  setDefault(outputDim -> 1)
+
+  setDefault(outputCol -> "lsh_output")
+
+  /**
+   * Transform the Schema for LSH
+   * @param schema The schema of the input dataset without outputCol
+   * @return A derived schema with outputCol added
+   */
+  final def transformLSHSchema(schema: StructType): StructType = {
+val outputFields = schema.fields :+
+  StructField($(outputCol), new VectorUDT, nullable = false)
+StructType(outputFields)
+  }
+}
+
+/**
+ * Model produced by [[LSH]].
+ */
+abstract class LSHModel[KeyType, T <: LSHModel[KeyType, T]] private[ml]
+  extends Model[T] with LSHParams {
+  override def copy(extra: ParamMap): T = defaultCopy(extra)
+  /**
+   * :: DeveloperApi ::
+   *
+   * The hash function of LSH, mapping a predefined KeyType to a Vector
+   * @return The mapping of LSH function.
+   */
+  protected[this] val hashFunction: KeyType => Vector
+
+  /**
+   * :: DeveloperApi ::
+   *
+   * Calculate the distance between two different keys using the distance 
metric corresponding
+   * to the hashFunction
+   * @param x One of the point in the metric space
+   * @param y Another the point in the metric space
+   * @return The distance between x and y in double
+   */
+  protected[ml] def keyDistance(x: KeyType, y: KeyType): Double
+
+  /**
+   * :: DeveloperApi ::
+   *
+   * Calculate the distance between two different hash Vectors. By 
default, the distance is the
+   * minimum distance of two hash values in any dimension.
+   *
+   * @param x One of the hash vector
+   * @param y Another hash vector
+   * @return The distance between hash vectors x and y in double
+   */
+  protected[ml] def hashDistance(x: Vector, y: Vector): Double = {
+// Since it's generated by hashing, it will be a pair of dense vectors.
+x.toDense.values.zip(y.toDense.values).map(x => math.abs(x._1 - 
x._2)).min
+  }
+
+  /**
+   * Transforms the input dataset.
+   */
+  override def transform(dataset: Dataset[_]): DataFrame = {
+if (!dataset.columns.contains($(outputCol))) {
+  transformSchema(dataset.schema, logging = true)
+  val transformUDF = udf(hashFunction, new VectorUDT)
+  dataset.withColumn($(outputCol), transformUDF(dataset($(inputCol
+} else {
+  dataset.toDF()
+}
+  }
+
+  /**
+   * :: DeveloperApi ::
+   *
+   * Check transform validity and derive the output schema from the input 
schema.
+   *
+   * Typical implementation should first conduct verification on schema 
change and parameter
+   * validity, including complex parameter interaction checks.
+   */
+  override def 

[GitHub] spark pull request #15244: Update spark-standalone.md to fix link

2016-09-26 Thread ammills01
GitHub user ammills01 opened a pull request:

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

Update spark-standalone.md to fix link

## What changes were proposed in this pull request?

Corrected a link to the configuration.html page, it was pointing to a page 
that does not exist (configurations.html).

## How was this patch tested?

Documentation change, verified in preview.

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

$ git pull https://github.com/ammills01/spark master

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

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


commit 7cc60c2a05d1f21449b0f592f89f31b13627627b
Author: Andrew Mills 
Date:   2016-09-26T19:56:45Z

Update spark-standalone.md to fix link

Corrected a link to the configuration.html page, it was pointing to a page 
that does not exist (configurations.html).




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15148: [SPARK-5992][ML] Locality Sensitive Hashing

2016-09-26 Thread jkbradley
Github user jkbradley commented on a diff in the pull request:

https://github.com/apache/spark/pull/15148#discussion_r80556472
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/lsh/LSH.scala ---
@@ -0,0 +1,290 @@
+/*
+ * 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.ml.feature.lsh
+
+import scala.util.Random
+
+import org.apache.spark.ml.{Estimator, Model}
+import org.apache.spark.ml.linalg.{Vector, VectorUDT}
+import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
+import org.apache.spark.sql._
+import org.apache.spark.sql.expressions.UserDefinedFunction
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.types._
+
+/**
+ * Params for [[LSH]].
+ */
+private[ml] trait LSHParams extends HasInputCol with HasOutputCol {
+  /**
+   * Param for output dimension.
+   *
+   * @group param
+   */
+  final val outputDim: IntParam = new IntParam(this, "outputDim", "output 
dimension",
+ParamValidators.gt(0))
+
+  /** @group getParam */
+  final def getOutputDim: Int = $(outputDim)
+
+  setDefault(outputDim -> 1)
+
+  setDefault(outputCol -> "lsh_output")
+
+  /**
+   * Transform the Schema for LSH
+   * @param schema The schema of the input dataset without outputCol
+   * @return A derived schema with outputCol added
+   */
+  final def transformLSHSchema(schema: StructType): StructType = {
+val outputFields = schema.fields :+
+  StructField($(outputCol), new VectorUDT, nullable = false)
+StructType(outputFields)
+  }
+}
+
+/**
+ * Model produced by [[LSH]].
+ */
+abstract class LSHModel[KeyType, T <: LSHModel[KeyType, T]] private[ml]
+  extends Model[T] with LSHParams {
+  override def copy(extra: ParamMap): T = defaultCopy(extra)
+  /**
+   * :: DeveloperApi ::
+   *
+   * The hash function of LSH, mapping a predefined KeyType to a Vector
+   * @return The mapping of LSH function.
+   */
+  protected[this] val hashFunction: KeyType => Vector
+
+  /**
+   * :: DeveloperApi ::
+   *
+   * Calculate the distance between two different keys using the distance 
metric corresponding
+   * to the hashFunction
+   * @param x One of the point in the metric space
+   * @param y Another the point in the metric space
+   * @return The distance between x and y in double
+   */
+  protected[ml] def keyDistance(x: KeyType, y: KeyType): Double
+
+  /**
+   * :: DeveloperApi ::
+   *
+   * Calculate the distance between two different hash Vectors. By 
default, the distance is the
+   * minimum distance of two hash values in any dimension.
+   *
+   * @param x One of the hash vector
+   * @param y Another hash vector
+   * @return The distance between hash vectors x and y in double
+   */
+  protected[ml] def hashDistance(x: Vector, y: Vector): Double = {
+// Since it's generated by hashing, it will be a pair of dense vectors.
+x.toDense.values.zip(y.toDense.values).map(x => math.abs(x._1 - 
x._2)).min
+  }
+
+  /**
+   * Transforms the input dataset.
+   */
+  override def transform(dataset: Dataset[_]): DataFrame = {
+transformSchema(dataset.schema, logging = true)
+val transformUDF = udf(hashFunction, new VectorUDT)
+dataset.withColumn($(outputCol), transformUDF(dataset($(inputCol
+  }
+
+  /**
+   * :: DeveloperApi ::
+   *
+   * Check transform validity and derive the output schema from the input 
schema.
+   *
+   * Typical implementation should first conduct verification on schema 
change and parameter
+   * validity, including complex parameter interaction checks.
+   */
+  override def transformSchema(schema: StructType): StructType = {
+transformLSHSchema(schema)
+  }
+
+  /**
+   * Given a 

[GitHub] spark pull request #14803: [SPARK-17153][SQL] Should read partition data whe...

2016-09-26 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/14803#discussion_r80556219
  
--- Diff: docs/structured-streaming-programming-guide.md ---
@@ -512,6 +512,12 @@ csvDF = spark \
 
 These examples generate streaming DataFrames that are untyped, meaning 
that the schema of the DataFrame is not checked at compile time, only checked 
at runtime when the query is submitted. Some operations like `map`, `flatMap`, 
etc. need the type to be known at compile time. To do those, you can convert 
these untyped streaming DataFrames to typed streaming Datasets using the same 
methods as static DataFrame. See the [SQL Programming 
Guide](sql-programming-guide.html) for more details. Additionally, more details 
on the supported streaming sources are discussed later in the document.
 
+### Schema inference and partition of streaming DataFrames/Datasets
+
+By default, Structured Streaming from file based sources requires you to 
specify the schema, rather than rely on Spark to infer it automatically. This 
restriction ensures a consistent schema will be used for the streaming query, 
even in the case of failures. For ad-hoc use cases, you can reenable schema 
inference by setting `spark.sql.streaming.schemaInference` to `true`.
--- End diff --

Schema inference can lead to many corner cases regarding if the inferred 
schema is different after restart. So I think we should use a stronger language 
that schema inference is not advisable in production uses.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15148: [SPARK-5992][ML] Locality Sensitive Hashing

2016-09-26 Thread jkbradley
Github user jkbradley commented on a diff in the pull request:

https://github.com/apache/spark/pull/15148#discussion_r80555938
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/lsh/LSH.scala ---
@@ -0,0 +1,290 @@
+/*
+ * 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.ml.feature.lsh
+
+import scala.util.Random
+
+import org.apache.spark.ml.{Estimator, Model}
+import org.apache.spark.ml.linalg.{Vector, VectorUDT}
+import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
+import org.apache.spark.sql._
+import org.apache.spark.sql.expressions.UserDefinedFunction
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.types._
+
+/**
+ * Params for [[LSH]].
+ */
+private[ml] trait LSHParams extends HasInputCol with HasOutputCol {
+  /**
+   * Param for output dimension.
+   *
+   * @group param
+   */
+  final val outputDim: IntParam = new IntParam(this, "outputDim", "output 
dimension",
+ParamValidators.gt(0))
+
+  /** @group getParam */
+  final def getOutputDim: Int = $(outputDim)
+
+  setDefault(outputDim -> 1)
+
+  setDefault(outputCol -> "lsh_output")
+
+  /**
+   * Transform the Schema for LSH
+   * @param schema The schema of the input dataset without outputCol
+   * @return A derived schema with outputCol added
+   */
+  final def transformLSHSchema(schema: StructType): StructType = {
+val outputFields = schema.fields :+
+  StructField($(outputCol), new VectorUDT, nullable = false)
+StructType(outputFields)
+  }
+}
+
+/**
+ * Model produced by [[LSH]].
+ */
+abstract class LSHModel[KeyType, T <: LSHModel[KeyType, T]] private[ml]
+  extends Model[T] with LSHParams {
+  override def copy(extra: ParamMap): T = defaultCopy(extra)
+  /**
+   * :: DeveloperApi ::
+   *
+   * The hash function of LSH, mapping a predefined KeyType to a Vector
+   * @return The mapping of LSH function.
+   */
+  protected[this] val hashFunction: KeyType => Vector
+
+  /**
+   * :: DeveloperApi ::
+   *
+   * Calculate the distance between two different keys using the distance 
metric corresponding
+   * to the hashFunction
+   * @param x One of the point in the metric space
+   * @param y Another the point in the metric space
+   * @return The distance between x and y in double
+   */
+  protected[ml] def keyDistance(x: KeyType, y: KeyType): Double
+
+  /**
+   * :: DeveloperApi ::
+   *
+   * Calculate the distance between two different hash Vectors. By 
default, the distance is the
+   * minimum distance of two hash values in any dimension.
+   *
+   * @param x One of the hash vector
+   * @param y Another hash vector
+   * @return The distance between hash vectors x and y in double
+   */
+  protected[ml] def hashDistance(x: Vector, y: Vector): Double = {
+// Since it's generated by hashing, it will be a pair of dense vectors.
+x.toDense.values.zip(y.toDense.values).map(x => math.abs(x._1 - 
x._2)).min
--- End diff --

If it's algorithm-specific, I'd recommend making it abstract here so it's 
more future bug-proof.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15196: [SPARK-17614] [SQL] sparkSession.read() .jdbc(***) use t...

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

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


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

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



[GitHub] spark issue #14897: [SPARK-17338][SQL] add global temp view

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #13680: [SPARK-15962][SQL] Introduce implementation with a dense...

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

https://github.com/apache/spark/pull/13680
  
**[Test build #65930 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65930/consoleFull)**
 for PR 13680 at commit 
[`2ef6e3b`](https://github.com/apache/spark/commit/2ef6e3bdbd16c7f7b9ff006d48382e108ed37eef).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #13680: [SPARK-15962][SQL] Introduce implementation with a dense...

2016-09-26 Thread kiszk
Github user kiszk commented on the issue:

https://github.com/apache/spark/pull/13680
  
Jenkins, 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 #14897: [SPARK-17338][SQL] add global temp view

2016-09-26 Thread yhuai
Github user yhuai commented on the issue:

https://github.com/apache/spark/pull/14897
  
test this please


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

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



[GitHub] spark issue #15196: [SPARK-17614] [SQL] sparkSession.read() .jdbc(***) use t...

2016-09-26 Thread srowen
Github user srowen commented on the issue:

https://github.com/apache/spark/pull/15196
  
Sounds fine. I guess I'm surprised these wouldn't have to be overridden in 
the same way, but, on the other hand I suppose the existing query works today 
for MySQL et al? this is a more conservative change now, yes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #13493: [SPARK-15750][MLLib][PYSPARK] Constructing FPGrowth fail...

2016-09-26 Thread jkbradley
Github user jkbradley commented on the issue:

https://github.com/apache/spark/pull/13493
  
Same here, looks fine except for @holdenk 's comment.  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 #14852: [SPARK-17138][ML][MLib] Add Python API for multinomial l...

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

https://github.com/apache/spark/pull/14852
  
LGTM. @yanboliang what do you think?


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

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



[GitHub] spark issue #15240: [SPARK-17556] Executor side broadcast for broadcast join...

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

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


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

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



[GitHub] spark issue #15240: [SPARK-17556] Executor side broadcast for broadcast join...

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

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


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

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



[GitHub] spark issue #15240: [SPARK-17556] Executor side broadcast for broadcast join...

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

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


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

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



[GitHub] spark issue #15196: [SPARK-17614] [SQL] sparkSession.read() .jdbc(***) use t...

2016-09-26 Thread JoshRosen
Github user JoshRosen commented on the issue:

https://github.com/apache/spark/pull/15196
  
@srowen, I think that @sureshthalamati is right and that we shouldn't 
override for those other dialects until we know that there's a problem with 
them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14079: [SPARK-8425][CORE] New Blacklist Mechanism

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

https://github.com/apache/spark/pull/14079
  
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 #14079: [SPARK-8425][CORE] New Blacklist Mechanism

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

https://github.com/apache/spark/pull/14079
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65922/
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 #14079: [SPARK-8425][CORE] New Blacklist Mechanism

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

https://github.com/apache/spark/pull/14079
  
**[Test build #65922 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65922/consoleFull)**
 for PR 14079 at commit 
[`2381b25`](https://github.com/apache/spark/commit/2381b2526cc2654258a85d496d9416ded06eeb20).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class ExecutorFailuresInTaskSet(val node: String) `


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

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



[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

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

https://github.com/apache/spark/pull/14079
  
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 #14079: [SPARK-8425][CORE] New Blacklist Mechanism

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

https://github.com/apache/spark/pull/14079
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65919/
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 #14818: [SPARK-17157][SPARKR][WIP]: Add multiclass logistic regr...

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

https://github.com/apache/spark/pull/14818
  
Could you elaborate what's incompatible?




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14079: [SPARK-8425][CORE] New Blacklist Mechanism

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

https://github.com/apache/spark/pull/14079
  
**[Test build #65919 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65919/consoleFull)**
 for PR 14079 at commit 
[`0c3ceba`](https://github.com/apache/spark/commit/0c3cebac02eeb194f1792db6235cffd50833a783).
 * 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 #15243: Fixing comment since Actor is not used anymore.

2016-09-26 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/15243
  
Hi, @danix800 .
Could you fix the following comment in this PR, too?


https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala#L256
```
// Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should 
throw exception.
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

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

https://github.com/apache/spark/pull/11601
  
So, I am trying to refresh my memory on this PR. I see we settled on not 
supporting vector type and not supporting mode. Did we ever settle on 
supporting multiple input columns? I am not sure I see the practicality of this 
if we don't. I wonder how many use cases fit the mold of imputing a single 
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 #11601: [SPARK-13568] [ML] Create feature transformer to ...

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

https://github.com/apache/spark/pull/11601#discussion_r80543997
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
@@ -0,0 +1,122 @@
+/*
+ * 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.ml.feature
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.util.{DefaultReadWriteTest}
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.mllib.util.TestingUtils._
+import org.apache.spark.sql.Row
+
+class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  test("Imputer for Double with default missing Value NaN") {
+val df = spark.createDataFrame( Seq(
+  (0, 1.0, 1.0, 1.0),
+  (1, 1.0, 1.0, 1.0),
+  (2, 3.0, 3.0, 3.0),
+  (3, 4.0, 4.0, 4.0),
+  (4, Double.NaN, 2.25, 1.0)
+)).toDF("id", "value", "expected_mean", "expected_median")
+Seq("mean", "median").foreach { strategy =>
+  val imputer = new 
Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
+  val model = imputer.fit(df)
+  model.transform(df).select("expected_" + strategy, 
"out").collect().foreach {
+   case Row(exp: Double, out: Double) =>
+  assert(exp ~== out absTol 1e-5, s"Imputed values differ. 
Expected: $exp, actual: $out")
+  }
+}
+  }
+
--- End diff --

Yeah, actually this also fails if the entire input column is the missing 
value as well. We need to beef up the test suite :)


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

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



[GitHub] spark issue #15239: [SPARK-17665][SPARKR][WIP] Support options/mode all for ...

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

https://github.com/apache/spark/pull/15239
  
also, you would need to add `@param ...` doc to pass the CRAN tests


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

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



[GitHub] spark issue #15239: [SPARK-17665][SPARKR][WIP] Support options/mode all for ...

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

https://github.com/apache/spark/pull/15239
  
great, thanks.
- we should consolidate the write.* function to use a helper to avoid code 
duplication
- I'm a bit worry about the function signature changes - could we have some 
tests for before/after?


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

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



[GitHub] spark pull request #15239: [SPARK-17665][SPARKR][WIP] Support options/mode a...

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

https://github.com/apache/spark/pull/15239#discussion_r80541497
  
--- Diff: R/pkg/R/SQLContext.R ---
@@ -835,7 +843,7 @@ loadDF <- function(x, ...) {
 #' @note createExternalTable since 1.4.0
 createExternalTable.default <- function(tableName, path = NULL, source = 
NULL, ...) {
   sparkSession <- getSparkSession()
-  options <- varargsToEnv(...)
--- End diff --

is there any use of `varargsToEnv` left?


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

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



[GitHub] spark pull request #15239: [SPARK-17665][SPARKR][WIP] Support options/mode a...

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

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

this might not be ideal because the user is not calling this function 
directly and `value[something]` might not mean anything to them (since they 
have never set any value thing, furthermore, that might not be the relevant 
syntax in R)

Any idea on a different way to report 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 #15239: [SPARK-17665][SPARKR][WIP] Support options/mode a...

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

https://github.com/apache/spark/pull/15239#discussion_r80540376
  
--- Diff: R/pkg/R/DataFrame.R ---
@@ -743,8 +743,12 @@ setMethod("toJSON",
 #' @note write.json since 1.6.0
 setMethod("write.json",
   signature(x = "SparkDataFrame", path = "character"),
-  function(x, path) {
+  function(x, path, mode = "error", ...) {
--- End diff --

does this change the default on the JVM side when mode was previously unset?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15216: [SPARK-17577][Follow-up][SparkR] SparkR spark.addFile su...

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

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


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

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



[GitHub] spark issue #14818: [SPARK-17157][SPARKR][WIP]: Add multiclass logistic regr...

2016-09-26 Thread wangmiao1981
Github user wangmiao1981 commented on the issue:

https://github.com/apache/spark/pull/14818
  
@felixcheung  If I added it to `spark.glm`, it will break the current 
`spark.glm` interface.

`setMethod("spark.glm", signature(data = "SparkDataFrame", formula = 
"formula"),
  function(data, formula, family = gaussian, tol = 1e-6, maxIter = 
25, weightCol = NULL,
   regParam = 0.0) `

With this interface, I can't add `LogisticRegressionModel`. I think it is 
better to add it to a separate Class, instead of the 
`GeneralizedLinearRegressionModel`.


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

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



[GitHub] spark issue #14897: [SPARK-17338][SQL] add global temp view

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

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


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

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



[GitHub] spark issue #14897: [SPARK-17338][SQL] add global temp view

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

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


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

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



[GitHub] spark issue #14897: [SPARK-17338][SQL] add global temp view

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

https://github.com/apache/spark/pull/14897
  
**[Test build #65923 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65923/consoleFull)**
 for PR 14897 at commit 
[`98d201b`](https://github.com/apache/spark/commit/98d201b71c5899d59ee3a63f0f153e70f9545308).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class SparkContext(config: SparkConf) extends Logging `
  * `// ('scaled' = +Infinity). However in the case that this class 
also has`
  * `// 0 probability, the class will not be selected ('scaled' is 
NaN).`
  * `  final val thresholds: DoubleArrayParam = new DoubleArrayParam(this, 
\"thresholds\", \"Thresholds in multi-class classification to adjust the 
probability of predicting each class. Array must have length equal to the 
number of classes, with values > 0 excepting that at most one value may be 0. 
The class with largest value p/t is predicted, where p is the original 
probability of that class and t is the class's threshold\", (t: Array[Double]) 
=> t.forall(_ >= 0) && t.count(_ == 0) <= 1)`
  * `thresholds = Param(Params._dummy(), \"thresholds\", \"Thresholds 
in multi-class classification to adjust the probability of predicting each 
class. Array must have length equal to the number of classes, with values > 0, 
excepting that at most one value may be 0. The class with largest value p/t is 
predicted, where p is the original probability of that class and t is the 
class's threshold.\", typeConverter=TypeConverters.toListFloat)`
  * `case class SortOrder(child: Expression, direction: SortDirection, 
nullOrdering: NullOrdering)`
  * `  case class FileEntry(path: String, timestamp: Timestamp, batchId: 
Long) extends Serializable`
  * `trait Offset 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 #14426: [SPARK-16475][SQL] Broadcast Hint for SQL Queries

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #15231: [SPARK-17658][SPARKR] read.df/write.df API taking...

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

https://github.com/apache/spark/pull/15231#discussion_r80536437
  
--- Diff: R/pkg/R/utils.R ---
@@ -698,6 +698,21 @@ isSparkRShell <- function() {
   grepl(".*shell\\.R$", Sys.getenv("R_PROFILE_USER"), perl = TRUE)
 }
 
+captureJVMException <- function(e) {
+  stacktrace <- as.character(e)
+  if (any(grep("java.lang.IllegalArgumentException: ", stacktrace))) {
--- End diff --

I agree, I don't think we should couple the R code to the underlining data 
source implementations, and was not suggesting that :)

I guess I'm saying there are still many (other) cases where the parameters 
are unchecked and would be good if this check to convert JVM 
IllegalArgumentException is sufficient or more checks should be added to the R 
side.


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

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



[GitHub] spark issue #14897: [SPARK-17338][SQL] add global temp view

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

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


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

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



[GitHub] spark issue #14897: [SPARK-17338][SQL] add global temp view

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

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


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

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



<    1   2   3   4   5   6   >