[GitHub] spark pull request #20616: [SPARK-23434][SQL] Spark should not warn `metadat...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20616#discussion_r169236998
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
 ---
@@ -42,9 +42,13 @@ object FileStreamSink extends Logging {
 try {
   val hdfsPath = new Path(singlePath)
   val fs = hdfsPath.getFileSystem(hadoopConf)
-  val metadataPath = new Path(hdfsPath, metadataDir)
-  val res = fs.exists(metadataPath)
-  res
+  if (fs.isDirectory(hdfsPath)) {
+val metadataPath = new Path(hdfsPath, metadataDir)
+val res = fs.exists(metadataPath)
+res
--- End diff --

nit: just `fs.exists(new Path(hdfsPath, metadataDir))`


---

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



[GitHub] spark pull request #20624: [SPARK-23445] ColumnStat refactoring

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20624#discussion_r169235761
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala
 ---
@@ -32,13 +32,18 @@ object AggregateEstimation {
 val childStats = agg.child.stats
 // Check if we have column stats for all group-by columns.
 val colStatsExist = agg.groupingExpressions.forall { e =>
-  e.isInstanceOf[Attribute] && 
childStats.attributeStats.contains(e.asInstanceOf[Attribute])
+  e.isInstanceOf[Attribute] && (
+childStats.attributeStats.get(e.asInstanceOf[Attribute]) match {
--- End diff --

nit: `childStats.attributeStats.get(e.asInstanceOf[Attribute]).exists(_. 
hasCountStats)`


---

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



[GitHub] spark pull request #20624: [SPARK-23445] ColumnStat refactoring

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20624#discussion_r169235322
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala
 ---
@@ -305,15 +260,15 @@ object ColumnStat extends Logging {
   percentiles: Option[ArrayData]): ColumnStat = {
 // The first 6 fields are basic column stats, the 7th is ndvs for 
histogram bins.
 val cs = ColumnStat(
-  distinctCount = BigInt(row.getLong(0)),
+  distinctCount = Option(BigInt(row.getLong(0))),
--- End diff --

nit: we should use `Some(value)` if `value` is expected to be not null.


---

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



[GitHub] spark pull request #20624: [SPARK-23445] ColumnStat refactoring

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20624#discussion_r169234962
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -387,6 +390,101 @@ case class CatalogStatistics(
   }
 }
 
+/**
+ * This class of statistics for a column is used in [[CatalogTable]] to 
interact with metastore.
+ */
+case class CatalogColumnStat(
+  distinctCount: Option[BigInt] = None,
+  min: Option[String] = None,
+  max: Option[String] = None,
+  nullCount: Option[BigInt] = None,
+  avgLen: Option[Long] = None,
+  maxLen: Option[Long] = None,
+  histogram: Option[Histogram] = None) {
+
+  /**
+   * Returns a map from string to string that can be used to serialize the 
column stats.
+   * The key is the name of the column and name of the field (e.g. 
"colName.distinctCount"),
+   * and the value is the string representation for the value.
+   * min/max values are stored as Strings. They can be deserialized using
+   * [[ColumnStat.fromExternalString]].
+   *
+   * As part of the protocol, the returned map always contains a key 
called "version".
+   * In the case min/max values are null (None), they won't appear in the 
map.
--- End diff --

now all fields are optional, we should update this comment.


---

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



[GitHub] spark issue #19788: [SPARK-9853][Core] Optimize shuffle fetch of contiguous ...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/19788
  
The idea LGTM, but I think @JoshRosen has a valid concern. My 2 cents:
1. The concept of reading multiple reducer partitions in one shot was 
introduced by `ShuffleManager.getReader`. Although it's only used for adaptive 
execution in Spark SQL, this is still a feature provided by Spark Core.
2. We should not force users to upgrade the external shuffle service when 
upgrading Spark, if they don't use adaptive execution.
3. We should not enable this batch shuffle fetching if the serializer and 
compressor don't support it.
4. For better user experience, we should avoid adding more configs if 
possible

My proposal: we should enable this feature only if
1. We need to fetch multiple reducer partitions at once, which can only 
happen when adaptive execution is enabled, currently.
2. The serializer supports it, i.e. 
`Serializer.supportsRelocationOfSerializedObjects` is true
3. The compressor supports it, i.e. 
`CompressionCodec.supportsConcatenationOfSerializedStreams` is true

Thus we don't need extra config and we will automatically enable it when 
adaptive execution is enabled.


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark pull request #19788: [SPARK-9853][Core] Optimize shuffle fetch of cont...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19788#discussion_r169229650
  
--- Diff: core/src/main/scala/org/apache/spark/TaskEndReason.scala ---
@@ -81,16 +81,17 @@ case object Resubmitted extends TaskFailedReason {
  */
 @DeveloperApi
 case class FetchFailed(
-bmAddress: BlockManagerId,  // Note that bmAddress can be null
+bmAddress: BlockManagerId, // Note that bmAddress can be null
 shuffleId: Int,
 mapId: Int,
 reduceId: Int,
-message: String)
+message: String,
+numBlocks: Int = 1)
   extends TaskFailedReason {
   override def toErrorString: String = {
 val bmAddressString = if (bmAddress == null) "null" else 
bmAddress.toString
 s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, 
reduceId=$reduceId, " +
-  s"message=\n$message\n)"
+  s"numBlocks=$numBlocks, message=\n$message\n)"
--- End diff --

do we really care about the `numBlock` when a shuffle fetch failed?


---

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



[GitHub] spark pull request #19788: [SPARK-9853][Core] Optimize shuffle fetch of cont...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19788#discussion_r169229539
  
--- Diff: core/src/main/scala/org/apache/spark/storage/BlockId.scala ---
@@ -51,11 +51,25 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) 
extends BlockId {
 
 // Format of the shuffle block ids (including data and index) should be 
kept in sync with
 // 
org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getBlockData().
+trait ShuffleBlockIdBase {
--- End diff --

we can make this extend `BlockId`.


---

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



[GitHub] spark pull request #19788: [SPARK-9853][Core] Optimize shuffle fetch of cont...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19788#discussion_r169229060
  
--- Diff: core/src/main/scala/org/apache/spark/MapOutputTracker.scala ---
@@ -280,6 +281,16 @@ private[spark] abstract class MapOutputTracker(conf: 
SparkConf) extends Logging
 }
   }
 
+  protected def supportsContinuousBlockBulkFetch: Boolean = {
+// continuousBlockBulkFetch only happens in SparkSQL, it uses 
UnsafeRowSerializer,
+// which supports relocation of serialized objects, so we only 
consider compression
+val adaptiveEnabled: Boolean = 
conf.getBoolean("spark.sql.adaptive.enabled", false)
--- End diff --

I don't think we can get this conf here. SQL conf is per-session and is 
kind of a layer above Spark conf.


---

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



[GitHub] spark pull request #19788: [SPARK-9853][Core] Optimize shuffle fetch of cont...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19788#discussion_r169228298
  
--- Diff: 
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
 ---
@@ -157,21 +157,34 @@ public void registerExecutor(
   }
 
   /**
-   * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). 
We make assumptions
+   * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId, 
numBlocks). We make assumptions
* about how the hash and sort based shuffles store their data.
*/
   public ManagedBuffer getBlockData(
   String appId,
   String execId,
   int shuffleId,
   int mapId,
-  int reduceId) {
+  int reduceId,
+  int numBlocks) {
 ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, 
execId));
 if (executor == null) {
   throw new RuntimeException(
 String.format("Executor is not registered (appId=%s, execId=%s)", 
appId, execId));
 }
-return getSortBasedShuffleBlockData(executor, shuffleId, mapId, 
reduceId);
+return getSortBasedShuffleBlockData(executor, shuffleId, mapId, 
reduceId, numBlocks);
+  }
+
+  /**
+   * This interface is for backward compatible.
+   */
+  public ManagedBuffer getBlockData(
--- End diff --

where will we call this method?


---

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



[GitHub] spark pull request #19788: [SPARK-9853][Core] Optimize shuffle fetch of cont...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19788#discussion_r169227734
  
--- Diff: 
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java
 ---
@@ -196,42 +196,51 @@ private ShuffleMetrics() {
 private final String appId;
 private final String execId;
 private final int shuffleId;
-// An array containing mapId and reduceId pairs.
-private final int[] mapIdAndReduceIds;
+// An array containing mapId, reduceId and numBlocks tuple
+private final int[] shuffleBlockIds;
 
 ManagedBufferIterator(String appId, String execId, String[] blockIds) {
   this.appId = appId;
   this.execId = execId;
   String[] blockId0Parts = blockIds[0].split("_");
-  if (blockId0Parts.length != 4 || 
!blockId0Parts[0].equals("shuffle")) {
+  // length == 4: ShuffleBlockId
+  // length == 5: ContinuousShuffleBlockId
+  if (!(blockId0Parts.length == 4 || blockId0Parts.length == 5) ||
+!blockId0Parts[0].equals("shuffle")) {
 throw new IllegalArgumentException("Unexpected shuffle block id 
format: " + blockIds[0]);
   }
   this.shuffleId = Integer.parseInt(blockId0Parts[1]);
-  mapIdAndReduceIds = new int[2 * blockIds.length];
+  shuffleBlockIds = new int[3 * blockIds.length];
   for (int i = 0; i < blockIds.length; i++) {
 String[] blockIdParts = blockIds[i].split("_");
-if (blockIdParts.length != 4 || 
!blockIdParts[0].equals("shuffle")) {
+if (!(blockIdParts.length == 4 || blockIdParts.length == 5) ||
+  !blockIdParts[0].equals("shuffle")) {
--- End diff --

shall we create a `boolean isShuffleBlock(String[] blockIdParts)`?


---

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



[GitHub] spark issue #20616: [SPARK-23434][SQL] Spark should not warn `metadata direc...

2018-02-19 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/20616
  
Thank you for review, @cloud-fan .


---

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



[GitHub] spark issue #20616: [SPARK-23434][SQL] Spark should not warn `metadata direc...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/20616
  
LGTM, cc @zsxwing 


---

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



[GitHub] spark issue #20612: [SPARK-23424][SQL]Add codegenStageId in comment

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on the issue:

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


---

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



[GitHub] spark pull request #20612: [SPARK-23424][SQL]Add codegenStageId in comment

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20612#discussion_r169224985
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -1226,14 +1226,24 @@ class CodegenContext {
 
   /**
* Register a comment and return the corresponding place holder
+   *
+   * @param placeholderId an optionally specified identifier for the 
comment's placeholder.
+   *  The caller should make sure this identifier is 
unique within the
+   *  compilation unit. If this argument is not 
specified, a fresh identifier
+   *  will be automatically created and used as the 
placeholder.
+   * @param force whether to force registering the comments
*/
-  def registerComment(text: => String): String = {
+   def registerComment(
+   text: => String,
+   placeholderId: String = "",
+   force: Boolean = false): String = {
 // By default, disable comments in generated code because computing 
the comments themselves can
 // be extremely expensive in certain cases, such as deeply-nested 
expressions which operate over
 // inputs with wide schemas. For more details on the performance 
issues that motivated this
 // flat, see SPARK-15680.
-if (SparkEnv.get != null && 
SparkEnv.get.conf.getBoolean("spark.sql.codegen.comments", false)) {
-  val name = freshName("c")
+if (force ||
+  SparkEnv.get != null && 
SparkEnv.get.conf.getBoolean("spark.sql.codegen.comments", false)) {
+  val name = if (placeholderId != "") placeholderId else freshName("c")
--- End diff --

although the caller should guarantee `placeholderId` is unique, shall we 
add a check here for safe? e.g. 
`assert(!placeHolderToComments.containsKey("placeholderId"))`


---

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



[GitHub] spark issue #20043: [SPARK-22856][SQL] Add wrappers for codegen output and n...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/20043
  
Spark 2.3 should be ready soon, let's resume this PR :) 


---

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



[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

2018-02-19 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/20619
  
Thank you all!


---

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



[GitHub] spark pull request #20043: [SPARK-22856][SQL] Add wrappers for codegen outpu...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20043#discussion_r169223825
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
 ---
@@ -75,7 +75,7 @@ case class BoundReference(ordinal: Int, dataType: 
DataType, nullable: Boolean)
  |$javaType ${ev.value} = ${ev.isNull} ? 
${ctx.defaultValue(dataType)} : ($value);
""".stripMargin)
   } else {
-ev.copy(code = s"$javaType ${ev.value} = $value;", isNull = 
"false")
+ev.copy(code = s"$javaType ${ev.value} = $value;", isNull = 
LiteralValue("false"))
--- End diff --

I think it should be useful to the `isNull` field.


---

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



[GitHub] spark pull request #20043: [SPARK-22856][SQL] Add wrappers for codegen outpu...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20043#discussion_r169223570
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
 ---
@@ -75,7 +75,7 @@ case class BoundReference(ordinal: Int, dataType: 
DataType, nullable: Boolean)
  |$javaType ${ev.value} = ${ev.isNull} ? 
${ctx.defaultValue(dataType)} : ($value);
""".stripMargin)
   } else {
-ev.copy(code = s"$javaType ${ev.value} = $value;", isNull = 
"false")
+ev.copy(code = s"$javaType ${ev.value} = $value;", isNull = 
LiteralValue("false"))
--- End diff --

nit: shall we introduce a `TrueLiteral` and `FalseLiteral`?


---

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



[GitHub] spark pull request #20621: [SPARK-23436][SQL] Infer partition as Date only i...

2018-02-19 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #20621: [SPARK-23436][SQL] Infer partition as Date only if it ca...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/20621
  
thanks, merging to master!


---

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



[GitHub] spark pull request #20619: [SPARK-23457][SQL] Register task completion liste...

2018-02-19 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

2018-02-19 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/20619
  
thanks, merging to master!


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169213260
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -521,7 +542,15 @@ case class Signum(child: Expression) extends 
UnaryMathExpression(math.signum, "S
 case class Sin(child: Expression) extends UnaryMathExpression(math.sin, 
"SIN")
 
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Returns the hyperbolic sine of `expr`.",
+  usage = """
+  _FUNC_(expr) - Returns hyperbolic sine of `expr`, as if computed by
--- End diff --

I think we need double spaces ahead - I think the key is consistency but 
less invasive change.


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169213138
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -262,6 +273,11 @@ case class Cos(child: Expression) extends 
UnaryMathExpression(math.cos, "COS")
 
 @ExpressionDescription(
   usage = "_FUNC_(expr) - Returns the hyperbolic cosine of `expr`.",
+  arguments =
+"""
--- End diff --

I mean ...

```
arguments = """
  Arguments:
* expr - hyperbolic angle.
""",
```

to be consistent other styles here. For example:


https://github.com/apache/spark/blob/5683984520cfe9e9acf49e47a84a56af155a8ad2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala#L156-L168


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169213439
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1313,131 +1313,165 @@ object functions {
   
//
 
   /**
-   * Computes the cosine inverse of the given value; the returned angle is 
in the range
-   * 0.0 through pi.
--- End diff --

I am sorry if I missed a discussion about this. Why don't we keep this 
simple description for this method rather than removing out?

If we go this way, we should update both `functions.py` and `functions.R`. 
If you strongly prefer this, I would like to do this separately in another PR 
for both together.


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20637
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/963/
Test PASSed.


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread kiszk
Github user kiszk commented on the issue:

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


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20637
  
**[Test build #87548 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87548/testReport)**
 for PR 20637 at commit 
[`e2e9e36`](https://github.com/apache/spark/commit/e2e9e3604284949d9d762274e2e1f55348851073).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20618
  
**[Test build #87547 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87547/testReport)**
 for PR 20618 at commit 
[`2586b0f`](https://github.com/apache/spark/commit/2586b0f25ae0e563374c03d1db37d08aaf0c8c08).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20618
  
**[Test build #87546 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87546/testReport)**
 for PR 20618 at commit 
[`168d7b3`](https://github.com/apache/spark/commit/168d7b3715c8ad19b68c9542ffdb90bcb126c312).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20637
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/962/
Test PASSed.


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20637: [SPARK-23466][SQL] Remove redundant null checks in gener...

2018-02-19 Thread kiszk
Github user kiszk commented on the issue:

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


---

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



[GitHub] spark issue #20632: [SPARK-3159] added subtree pruning in the translation fr...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20632
  
**[Test build #4100 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4100/testReport)**
 for PR 20632 at commit 
[`fd3b5df`](https://github.com/apache/spark/commit/fd3b5df05352fa581780f17510bb0d6662359647).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20632: [SPARK-3159] added subtree pruning in the translation fr...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20632
  
**[Test build #4100 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4100/testReport)**
 for PR 20632 at commit 
[`fd3b5df`](https://github.com/apache/spark/commit/fd3b5df05352fa581780f17510bb0d6662359647).


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20618
  
**[Test build #87547 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87547/testReport)**
 for PR 20618 at commit 
[`2586b0f`](https://github.com/apache/spark/commit/2586b0f25ae0e563374c03d1db37d08aaf0c8c08).


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread misutoth
Github user misutoth commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169187041
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -804,7 +858,6 @@ case class Pow(left: Expression, right: Expression)
   }
 }
 
-
--- End diff --

Sure


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread misutoth
Github user misutoth commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169186706
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -538,8 +559,14 @@ case class Sinh(child: Expression) extends 
UnaryMathExpression(math.sinh, "SINH"
   """)
 case class Sqrt(child: Expression) extends UnaryMathExpression(math.sqrt, 
"SQRT")
 
+// scalastyle:off line.size.limit
--- End diff --

Turned it on for all trigonometric functions.


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread misutoth
Github user misutoth commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169186651
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -512,16 +522,27 @@ case class Rint(child: Expression) extends 
UnaryMathExpression(math.rint, "ROUND
 case class Signum(child: Expression) extends 
UnaryMathExpression(math.signum, "SIGNUM")
 
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Returns the sine of `expr`.",
+  usage = "_FUNC_(expr) - Returns the sine of `expr`, as if computed by 
`java.lang.Math._FUNC_`.",
+  arguments =
+"""
--- End diff --

Yes, I applied to all places in scope.


---

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



[GitHub] spark issue #20638: [SPARK-23417][python] Fix the build instructions supplie...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20638: [SPARK-23417][python] Fix the build instructions supplie...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20638: [SPARK-23417][python] Fix the build instructions supplie...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20638
  
**[Test build #87545 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87545/testReport)**
 for PR 20638 at commit 
[`8241313`](https://github.com/apache/spark/commit/824131355cfdd531101308cc6f803820aa5c941e).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20618
  
**[Test build #87546 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87546/testReport)**
 for PR 20618 at commit 
[`168d7b3`](https://github.com/apache/spark/commit/168d7b3715c8ad19b68c9542ffdb90bcb126c312).


---

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



[GitHub] spark issue #20638: [SPARK-23417][python] Fix the build instructions supplie...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20638
  
**[Test build #87545 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87545/testReport)**
 for PR 20638 at commit 
[`8241313`](https://github.com/apache/spark/commit/824131355cfdd531101308cc6f803820aa5c941e).


---

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



[GitHub] spark issue #20638: [SPARK-23417][python] Fix the build instructions supplie...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

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


---

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



[GitHub] spark issue #20638: [SPARK-23417][python] Fix the build instructions supplie...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/20638
  
add to whitelist


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

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


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169182568
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -804,7 +858,6 @@ case class Pow(left: Expression, right: Expression)
   }
 }
 
-
--- End diff --

Seems unrelated change. Could we just revert this back?


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169182146
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -512,16 +522,27 @@ case class Rint(child: Expression) extends 
UnaryMathExpression(math.rint, "ROUND
 case class Signum(child: Expression) extends 
UnaryMathExpression(math.signum, "SIGNUM")
 
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Returns the sine of `expr`.",
+  usage = "_FUNC_(expr) - Returns the sine of `expr`, as if computed by 
`java.lang.Math._FUNC_`.",
+  arguments =
+"""
--- End diff --

Can we have the same format for it and the same instances - 
https://github.com/apache/spark/pull/20618#discussion_r168670009?


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169182499
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -562,7 +595,12 @@ case class Cot(child: Expression)
 }
 
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Returns the hyperbolic tangent of `expr`.",
+  usage = "_FUNC_(expr) - Returns the hyperbolic tangent of `expr`, as if 
computed by `java.lang.Math._FUNC_`.",
--- End diff --

Seems line limit is reached here.


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169182420
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -548,7 +575,13 @@ case class Sqrt(child: Expression) extends 
UnaryMathExpression(math.sqrt, "SQRT"
 case class Tan(child: Expression) extends UnaryMathExpression(math.tan, 
"TAN")
 
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Returns the cotangent of `expr`.",
+  usage = "_FUNC_(expr) - Returns the cotangent of `expr` ," +
--- End diff --

Ditto for formatting


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169182396
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -538,8 +559,14 @@ case class Sinh(child: Expression) extends 
UnaryMathExpression(math.sinh, "SINH"
   """)
 case class Sqrt(child: Expression) extends UnaryMathExpression(math.sqrt, 
"SQRT")
 
+// scalastyle:off line.size.limit
--- End diff --

Ditto for turning it on


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169182053
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -512,16 +522,27 @@ case class Rint(child: Expression) extends 
UnaryMathExpression(math.rint, "ROUND
 case class Signum(child: Expression) extends 
UnaryMathExpression(math.signum, "SIGNUM")
 
 @ExpressionDescription(
-  usage = "_FUNC_(expr) - Returns the sine of `expr`.",
+  usage = "_FUNC_(expr) - Returns the sine of `expr`, as if computed by 
`java.lang.Math._FUNC_`.",
+  arguments =
+"""
+Arguments:
+  * expr - angle in radians
+""",
   examples = """
 Examples:
   > SELECT _FUNC_(0);
0.0
   """)
 case class Sin(child: Expression) extends UnaryMathExpression(math.sin, 
"SIN")
 
+// scalastyle:off line.size.limit
--- End diff --

I think we should turn this on again


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169182585
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -916,7 +969,6 @@ case class ShiftRightUnsigned(left: Expression, right: 
Expression)
 case class Hypot(left: Expression, right: Expression)
   extends BinaryMathExpression(math.hypot, "HYPOT")
 
-
--- End diff --

Ditto for revert


---

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



[GitHub] spark issue #20637: Remove redundant null checks in generated Java code by G...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20637: Remove redundant null checks in generated Java code by G...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20637: Remove redundant null checks in generated Java code by G...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20637
  
**[Test build #87544 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87544/testReport)**
 for PR 20637 at commit 
[`e2e9e36`](https://github.com/apache/spark/commit/e2e9e3604284949d9d762274e2e1f55348851073).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark pull request #20632: [SPARK-3159] added subtree pruning in the transla...

2018-02-19 Thread asolimando
Github user asolimando commented on a diff in the pull request:

https://github.com/apache/spark/pull/20632#discussion_r169172406
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala ---
@@ -402,20 +406,40 @@ class RandomForestSuite extends SparkFunSuite with 
MLlibTestSparkContext {
   LabeledPoint(1.0, Vectors.dense(2.0)))
 val input = sc.parallelize(arr)
 
+val seed = 42
+val numTrees = 1
+
 // Must set maxBins s.t. the feature will be treated as an ordered 
categorical feature.
 val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity 
= Gini, maxDepth = 1,
   numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3), maxBins = 3)
 
-val model = RandomForest.run(input, strategy, numTrees = 1, 
featureSubsetStrategy = "all",
-  seed = 42, instr = None).head
-model.rootNode match {
-  case n: InternalNode => n.split match {
-case s: CategoricalSplit =>
-  assert(s.leftCategories === Array(1.0))
-case _ => throw new AssertionError("model.rootNode.split was not a 
CategoricalSplit")
-  }
-  case _ => throw new AssertionError("model.rootNode was not an 
InternalNode")
-}
+val metadata = DecisionTreeMetadata.buildMetadata(input, strategy, 
numTrees = numTrees,
+  featureSubsetStrategy = "all")
+val splits = RandomForest.findSplits(input, metadata, seed = seed)
+
+val treeInput = TreePoint.convertToTreeRDD(input, splits, metadata)
+val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput,
+  strategy.subsamplingRate, numTrees, false, seed = seed)
+
+val topNode = LearningNode.emptyNode(nodeIndex = 1)
+assert(topNode.isLeaf === false)
+assert(topNode.stats === null)
+
+val nodesForGroup = Map((0, Array(topNode)))
+val treeToNodeToIndexInfo = Map((0, Map(
--- End diff --

I have updated the syntax, not only in the tests I have moved here myself, 
but everywhere in "ml/tree/impl/RandomForestSuite.scala", in order to have a 
uniform style at least throughout this file.


---

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



[GitHub] spark pull request #20632: [SPARK-3159] added subtree pruning in the transla...

2018-02-19 Thread asolimando
Github user asolimando commented on a diff in the pull request:

https://github.com/apache/spark/pull/20632#discussion_r169172053
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala ---
@@ -287,6 +291,34 @@ private[tree] class LearningNode(
 }
   }
 
+  /**
+   * @return true iff the node is a leaf.
+   */
+  private def isLeafNode(): Boolean = leftChild.isEmpty && 
rightChild.isEmpty
+
+  // the set of (leaf) predictions appearing in the subtree rooted at the 
given node.
+  private lazy val leafPredictions: Set[Double] = {
+
+val predBuffer = new scala.collection.mutable.HashSet[Double]
+
+// collect the (leaf) predictions in the left subtree, if any
+if (leftChild.isDefined) {
+  predBuffer ++= leftChild.get.leafPredictions
+}
+
+// collect the (leaf) predictions in the right subtree, if any
+if (predBuffer.size <= 1 && rightChild.isDefined) {
+  predBuffer ++= rightChild.get.leafPredictions
+}
+
+// if the node is a leaf, collect its prediction
+if (predBuffer.isEmpty) {
--- End diff --

Yes, it is correct.


---

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



[GitHub] spark issue #20638: [SPARK-23417][python] Fix the build instructions supplie...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20638: [SPARK-23417][python] Fix the build instructions supplie...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark pull request #20638: [SPARK-23417][python] Fix the build instructions ...

2018-02-19 Thread bersprockets
GitHub user bersprockets opened a pull request:

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

[SPARK-23417][python] Fix the build instructions supplied by exception 
messages in python streaming tests

## What changes were proposed in this pull request?

Fix the build instructions supplied by exception messages in python 
streaming tests. 

I also added -DskipTests to the maven instructions to avoid the 170 minutes 
of scala tests that occurs each time one wants to add a jar to the assembly 
directory.

## How was this patch tested?

- clone branch
- run build/sbt package
- run python/run-tests --modules "pyspark-streaming" , expect error message
- follow instructions in error message. i.e., run build/sbt 
assembly/package streaming-kafka-0-8-assembly/assembly
- rerun python tests, expect error message
- follow instructions in error message. i.e run build/sbt -Pflume 
assembly/package streaming-flume-assembly/assembly
- rerun python tests, see success.
- repeated all of the above for mvn version of the process.



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

$ git pull https://github.com/bersprockets/spark SPARK-23417_propa

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

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


commit a80a4c5df8064934784091e8650b8d13d09ef66f
Author: Bruce Robbins 
Date:   2018-02-17T17:30:06Z

Fix build instructions embedded in error message

commit 824131355cfdd531101308cc6f803820aa5c941e
Author: Bruce Robbins 
Date:   2018-02-17T18:55:17Z

Fix mvn instructions also




---

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



[GitHub] spark issue #20637: Remove redundant null checks in generated Java code by G...

2018-02-19 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20637: Remove redundant null checks in generated Java code by G...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20637: Remove redundant null checks in generated Java code by G...

2018-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20637
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/961/
Test PASSed.


---

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



[GitHub] spark pull request #20627: [SPARK-23217][ML][PYTHON] Add distanceMeasure par...

2018-02-19 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/20627#discussion_r169156086
  
--- Diff: python/pyspark/ml/evaluation.py ---
@@ -394,15 +397,30 @@ def getMetricName(self):
 @keyword_only
 @since("2.3.0")
 def setParams(self, predictionCol="prediction", featuresCol="features",
-  metricName="silhouette"):
+  metricName="silhouette", 
distanceMeasure="squaredEuclidean"):
--- End diff --

Yeah these will not cause any incompatibility issues, the interpreter would 
automatically just pass the additional keyword arg.  These methods are actually 
all wrapped with `keyword_only` which make it look externally like 
`setParams(**kwargs)`, anyway.


---

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



[GitHub] spark pull request #20627: [SPARK-23217][ML][PYTHON] Add distanceMeasure par...

2018-02-19 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/20627#discussion_r169156415
  
--- Diff: python/pyspark/ml/tests.py ---
@@ -541,6 +541,16 @@ def test_java_params(self):
 self.assertEqual(evaluator._java_obj.getMetricName(), "r2")
 self.assertEqual(evaluatorCopy._java_obj.getMetricName(), "mae")
 
+def test_clustering_evaluator_with_cosine_distance(self):
+featureAndPredictions = map(lambda x: (Vectors.dense(x[0]), x[1]),
+[([1.0, 1.0], 1.0), ([10.0, 10.0], 
1.0), ([1.0, 0.5], 2.0),
+ ([10.0, 4.4], 2.0), ([-1.0, 1.0], 
3.0), ([-100.0, 90.0], 3.0)])
+dataset = self.spark.createDataFrame(featureAndPredictions, 
["features", "prediction"])
+evaluator = ClusteringEvaluator(predictionCol="prediction", 
distanceMeasure="cosine")
+self.assertEqual(evaluator.getDistanceMeasure(), "cosine")
+self.assertEqual(round(evaluator.evaluate(dataset), 5),  0.99267)
--- End diff --

it would be better to use `np.allclose` rather than rounding.  Check out 
some other tests here for the usage


---

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



[GitHub] spark pull request #20627: [SPARK-23217][ML][PYTHON] Add distanceMeasure par...

2018-02-19 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/20627#discussion_r169158234
  
--- Diff: python/pyspark/ml/tests.py ---
@@ -541,6 +541,16 @@ def test_java_params(self):
 self.assertEqual(evaluator._java_obj.getMetricName(), "r2")
 self.assertEqual(evaluatorCopy._java_obj.getMetricName(), "mae")
 
+def test_clustering_evaluator_with_cosine_distance(self):
+featureAndPredictions = map(lambda x: (Vectors.dense(x[0]), x[1]),
+[([1.0, 1.0], 1.0), ([10.0, 10.0], 
1.0), ([1.0, 0.5], 2.0),
+ ([10.0, 4.4], 2.0), ([-1.0, 1.0], 
3.0), ([-100.0, 90.0], 3.0)])
+dataset = self.spark.createDataFrame(featureAndPredictions, 
["features", "prediction"])
+evaluator = ClusteringEvaluator(predictionCol="prediction", 
distanceMeasure="cosine")
+self.assertEqual(evaluator.getDistanceMeasure(), "cosine")
+self.assertEqual(round(evaluator.evaluate(dataset), 5),  0.99267)
+self.assertEqual(evaluator._java_obj.getDistanceMeasure(), 
"cosine")
--- End diff --

I don't think it is great to check the parameter in the java obj too.  If 
this was done for every param, it would be a ton of check and make it 
impossible to possibly change this plumbing in the future.

There is an existing test that runs some basic param checks that should be 
sufficient, but it doesn't look like the evaluation module is part of it, can 
you add it?  It's in `DefaultValuesTests.test_java_params`


---

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



[GitHub] spark pull request #20627: [SPARK-23217][ML][PYTHON] Add distanceMeasure par...

2018-02-19 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/20627#discussion_r169156894
  
--- Diff: python/pyspark/ml/tests.py ---
@@ -52,7 +52,7 @@
 from pyspark.ml.clustering import *
 from pyspark.ml.common import _java2py, _py2java
 from pyspark.ml.evaluation import BinaryClassificationEvaluator, \
-MulticlassClassificationEvaluator, RegressionEvaluator
+MulticlassClassificationEvaluator, RegressionEvaluator, 
ClusteringEvaluator
--- End diff --

fix import ordering


---

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



[GitHub] spark pull request #20637: Remove redundant null checks in generated Java co...

2018-02-19 Thread kiszk
GitHub user kiszk opened a pull request:

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

Remove redundant null checks in generated Java code by 
GenerateUnsafeProjection

## What changes were proposed in this pull request?

This PR works for one of TODOs in `GenerateUnsafeProjection` "if the 
nullability of field is correct, we can use it to save null check" to simplify 
generated code.  
When `nullable=false` in `DataType`, `GenerateUnsafeProjection` removed 
code for null checks in the generated Java code.

The following is an example.

Source code
```
val dataType3 = (new StructType)
  .add("a", StringType, nullable = false)
  .add("b", StringType, nullable = false)
  .add("c", StringType, nullable = false)
val exprs3 = BoundReference(0, dataType3, nullable = false) :: Nil
val projection3 = GenerateUnsafeProjection.generate(exprs3)
projection3.apply(InternalRow(AlwaysNonNull))
```
Generated code without this PR
```
/* 001 */ public java.lang.Object generate(Object[] references) {
/* 002 */   return new SpecificUnsafeProjection(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificUnsafeProjection extends 
org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder[] 
mutableStateArray1 = new 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder[1];
/* 009 */   private 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] 
mutableStateArray2 = new 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2];
/* 010 */   private UnsafeRow[] mutableStateArray = new UnsafeRow[1];
/* 011 */
/* 012 */   public SpecificUnsafeProjection(Object[] references) {
/* 013 */ this.references = references;
/* 014 */ mutableStateArray[0] = new UnsafeRow(1);
/* 015 */ mutableStateArray1[0] = new 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mutableStateArray[0],
 32);
/* 016 */ mutableStateArray2[0] = new 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mutableStateArray1[0],
 1);
/* 017 */ mutableStateArray2[1] = new 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mutableStateArray1[0],
 3);
/* 018 */
/* 019 */   }
/* 020 */
/* 021 */   public void initialize(int partitionIndex) {
/* 022 */
/* 023 */   }
/* 024 */
/* 025 */   // Scala.Function1 need this
/* 026 */   public java.lang.Object apply(java.lang.Object row) {
/* 027 */ return apply((InternalRow) row);
/* 028 */   }
/* 029 */
/* 030 */   public UnsafeRow apply(InternalRow i) {
/* 031 */ mutableStateArray1[0].reset();
/* 032 */
/* 033 */ InternalRow value = i.getStruct(0, 3);
/* 034 */ // Remember the current cursor so that we can calculate how 
many bytes are
/* 035 */ // written later.
/* 036 */ final int tmpCursor = mutableStateArray1[0].cursor;
/* 037 */
/* 038 */ final InternalRow tmpInput = value;
/* 039 */ if (tmpInput instanceof UnsafeRow) {
/* 040 */
/* 041 */   final int sizeInBytes = ((UnsafeRow) 
tmpInput).getSizeInBytes();
/* 042 */   // grow the global buffer before writing data.
/* 043 */   mutableStateArray1[0].grow(sizeInBytes);
/* 044 */   ((UnsafeRow) 
tmpInput).writeToMemory(mutableStateArray1[0].buffer, 
mutableStateArray1[0].cursor);
/* 045 */   mutableStateArray1[0].cursor += sizeInBytes;
/* 046 */
/* 047 */ } else {
/* 048 */   mutableStateArray2[1].reset();
/* 049 */
/* 050 */
/* 051 */   if (tmpInput.isNullAt(0)) {
/* 052 */ mutableStateArray2[1].setNullAt(0);
/* 053 */   } else {
/* 054 */ mutableStateArray2[1].write(0, tmpInput.getUTF8String(0));
/* 055 */   }
/* 056 */
/* 057 */
/* 058 */   if (tmpInput.isNullAt(1)) {
/* 059 */ mutableStateArray2[1].setNullAt(1);
/* 060 */   } else {
/* 061 */ mutableStateArray2[1].write(1, tmpInput.getUTF8String(1));
/* 062 */   }
/* 063 */
/* 064 */
/* 065 */   if (tmpInput.isNullAt(2)) {
/* 066 */ mutableStateArray2[1].setNullAt(2);
/* 067 */   } else {
/* 068 */ mutableStateArray2[1].write(2, tmpInput.getUTF8String(2));
/* 069 */   }
/* 070 */ }
/* 071 */
/* 072 */ mutableStateArray2[0].setOffsetAndSize(0, tmpCursor, 
mutableStateArray1[0].cursor - tmpCursor);
/* 073 */ 
mutableStateArray[0].setTotalSize(mutableStateArray1[0].totalSize());
/* 074 */ return mutableStateArray[0];
/* 075 */   }
/* 076 */
/* 077 */
/* 078 */ }
```

Generated code with this PR
```

[GitHub] spark pull request #20636: [SPARK-23415][SQL][TEST] Make behavior of BufferH...

2018-02-19 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/20636#discussion_r169151430
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala
 ---
@@ -58,15 +58,20 @@ object BufferHolderSparkSubmitSuite {
 val holder = new BufferHolder(new UnsafeRow(1000))
 
 holder.reset()
+// execute here since reset() updates holder.cursor
+val smallBuffer = new Array[Byte](holder.cursor)
--- End diff --

Ping, @liufengdb and @gatorsmile .


---

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



[GitHub] spark issue #20636: [SPARK-23415][SQL][TEST] Make behavior of BufferHolderSp...

2018-02-19 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/20636
  
Thank you for pining me and working on this issue, @kiszk .


---

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



[GitHub] spark pull request #20631: [SPARK-23454][SS][DOCS] Added trigger information...

2018-02-19 Thread bersprockets
Github user bersprockets commented on a diff in the pull request:

https://github.com/apache/spark/pull/20631#discussion_r169146175
  
--- Diff: docs/structured-streaming-programming-guide.md ---
@@ -1051,6 +1053,16 @@ from the aggregation column.
 For example, `df.groupBy("time").count().withWatermark("time", "1 min")` 
is invalid in Append 
 output mode.
 
+# Semantic Guarantees of Aggregation with Watermarking
+{:.no_toc}
+
+- A watermark delay (set with `withWatermark`) of "2 hours" guarantees 
that the engine will never
+drop any data that is less than 2 hours delayed. In other words, any data 
less than 2 hours behind
+(in terms of event-time) the latest data processed till then is guaranteed 
to be aggregated.
+
+- However, the guarantee is strict only in one direction. Data delayed by 
more than 2 hours is
+not guaranteed to be dropped; it may or may not get aggregated. More 
delayed is the data, less
+likely is the engine going to process it.
--- End diff --

> However, the guarantee is strict only in one direction. Data delayed by 
more than 2 hours is not guaranteed to be dropped

This might contradict an earlier statement, from "Handling Late Data and 
Watermarking", that says

"In other words, late data within the threshold will be aggregated, but 
data later than the threshold will be dropped"


---

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



[GitHub] spark issue #19881: [SPARK-22683][CORE] Add tasksPerExecutorSlot parameter

2018-02-19 Thread tgravescs
Github user tgravescs commented on the issue:

https://github.com/apache/spark/pull/19881
  
ping @jcuquemelle  can you update this?


---

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



[GitHub] spark issue #20571: [SPARK-23383][Build][Minor]Make a distribution should ex...

2018-02-19 Thread yaooqinn
Github user yaooqinn commented on the issue:

https://github.com/apache/spark/pull/20571
  
yes, single dash means the coming ones are all maven options,they will 
be handled later by mvn command. we do not parse each single option in this 
while loop from then till now. 


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169120482
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -770,7 +837,14 @@ case class Unhex(child: Expression) extends 
UnaryExpression with ImplicitCastInp
 
 // scalastyle:off line.size.limit
 @ExpressionDescription(
-  usage = "_FUNC_(expr1, expr2) - Returns the angle in radians between the 
positive x-axis of a plane and the point given by the coordinates (`expr1`, 
`expr2`).",
--- End diff --

Oh, I get it now, that's my mistake. "Binary" means "two arg" and not 
perhaps "bitwise" or something. Leave it then.


---

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



[GitHub] spark pull request #20627: [SPARK-23217][ML][PYTHON] Add distanceMeasure par...

2018-02-19 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/20627#discussion_r169116845
  
--- Diff: python/pyspark/ml/evaluation.py ---
@@ -394,15 +397,30 @@ def getMetricName(self):
 @keyword_only
 @since("2.3.0")
 def setParams(self, predictionCol="prediction", featuresCol="features",
-  metricName="silhouette"):
+  metricName="silhouette", 
distanceMeasure="squaredEuclidean"):
--- End diff --

I am not expert enough in Python to answer this. A normal python script is 
interpreted at runtime, so source code compatibility should be enough, but a 
`.pyc` file might have the same issue in theory.
I'd love to hear the answer from someone with more experience on Python.


---

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



[GitHub] spark issue #20618: [SPARK-23329][SQL] Fix documentation of trigonometric fu...

2018-02-19 Thread misutoth
Github user misutoth commented on the issue:

https://github.com/apache/spark/pull/20618
  
Thanks @srowen and @HyukjinKwon for your comments so far ...


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread misutoth
Github user misutoth commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169100931
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -770,7 +837,14 @@ case class Unhex(child: Expression) extends 
UnaryExpression with ImplicitCastInp
 
 // scalastyle:off line.size.limit
 @ExpressionDescription(
-  usage = "_FUNC_(expr1, expr2) - Returns the angle in radians between the 
positive x-axis of a plane and the point given by the coordinates (`expr1`, 
`expr2`).",
--- End diff --

Now I got it. Sorry, earlier I just misunderstood it. Are you proposing to 
move this case class inside the file to be next to the other trigonometric 
function? Based on the comments it seems the functions are group based on the 
number of arguments. This function has 2 arguments that is why it is under `// 
Binary math functions` section. Even though this does not seem to create big 
cohesion inside such a group I guess we do not want to reorganize the file now?


---

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



[GitHub] spark issue #20632: [SPARK-3159] added subtree pruning in the translation fr...

2018-02-19 Thread srowen
Github user srowen commented on the issue:

https://github.com/apache/spark/pull/20632
  
Another one that @jkbradley or @MLnick might want to look at, but seems 
like a nice win.


---

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



[GitHub] spark pull request #20632: [SPARK-3159] added subtree pruning in the transla...

2018-02-19 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/20632#discussion_r169091069
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala ---
@@ -287,6 +291,34 @@ private[tree] class LearningNode(
 }
   }
 
+  /**
+   * @return true iff the node is a leaf.
+   */
+  private def isLeafNode(): Boolean = leftChild.isEmpty && 
rightChild.isEmpty
+
+  // the set of (leaf) predictions appearing in the subtree rooted at the 
given node.
+  private lazy val leafPredictions: Set[Double] = {
+
+val predBuffer = new scala.collection.mutable.HashSet[Double]
+
+// collect the (leaf) predictions in the left subtree, if any
+if (leftChild.isDefined) {
+  predBuffer ++= leftChild.get.leafPredictions
+}
+
+// collect the (leaf) predictions in the right subtree, if any
+if (predBuffer.size <= 1 && rightChild.isDefined) {
+  predBuffer ++= rightChild.get.leafPredictions
+}
+
+// if the node is a leaf, collect its prediction
+if (predBuffer.isEmpty) {
--- End diff --

BTW is the logic here that you don't need to add the node's prediction if 
it's not a leaf, because its prediction must be contained in one of its 
children, and those predictions were already added?


---

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



[GitHub] spark pull request #20632: [SPARK-3159] added subtree pruning in the transla...

2018-02-19 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/20632#discussion_r169092238
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala ---
@@ -402,20 +406,40 @@ class RandomForestSuite extends SparkFunSuite with 
MLlibTestSparkContext {
   LabeledPoint(1.0, Vectors.dense(2.0)))
 val input = sc.parallelize(arr)
 
+val seed = 42
+val numTrees = 1
+
 // Must set maxBins s.t. the feature will be treated as an ordered 
categorical feature.
 val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity 
= Gini, maxDepth = 1,
   numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3), maxBins = 3)
 
-val model = RandomForest.run(input, strategy, numTrees = 1, 
featureSubsetStrategy = "all",
-  seed = 42, instr = None).head
-model.rootNode match {
-  case n: InternalNode => n.split match {
-case s: CategoricalSplit =>
-  assert(s.leftCategories === Array(1.0))
-case _ => throw new AssertionError("model.rootNode.split was not a 
CategoricalSplit")
-  }
-  case _ => throw new AssertionError("model.rootNode was not an 
InternalNode")
-}
+val metadata = DecisionTreeMetadata.buildMetadata(input, strategy, 
numTrees = numTrees,
+  featureSubsetStrategy = "all")
+val splits = RandomForest.findSplits(input, metadata, seed = seed)
+
+val treeInput = TreePoint.convertToTreeRDD(input, splits, metadata)
+val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput,
+  strategy.subsamplingRate, numTrees, false, seed = seed)
+
+val topNode = LearningNode.emptyNode(nodeIndex = 1)
+assert(topNode.isLeaf === false)
+assert(topNode.stats === null)
+
+val nodesForGroup = Map((0, Array(topNode)))
+val treeToNodeToIndexInfo = Map((0, Map(
--- End diff --

Nit: I think `Map(a -> b)` syntax is clearer than `Map((a, b))` syntax. 
(You use the former a few lines later too.)


---

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



[GitHub] spark issue #20636: [SPARK-23415][SQL][TEST] Make behavior of BufferHolderSp...

2018-02-19 Thread kiszk
Github user kiszk commented on the issue:

https://github.com/apache/spark/pull/20636
  
cc @dongjoon-hyun


---

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



[GitHub] spark pull request #20627: [SPARK-23217][ML][PYTHON] Add distanceMeasure par...

2018-02-19 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/20627#discussion_r169090021
  
--- Diff: python/pyspark/ml/evaluation.py ---
@@ -394,15 +397,30 @@ def getMetricName(self):
 @keyword_only
 @since("2.3.0")
 def setParams(self, predictionCol="prediction", featuresCol="features",
-  metricName="silhouette"):
+  metricName="silhouette", 
distanceMeasure="squaredEuclidean"):
--- End diff --

Just so I'm clear, this doesn't cause any API incompatibility in Python in 
the way it would for Scala right?


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169086289
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1313,131 +1313,168 @@ object functions {
   
//
 
   /**
-   * Computes the cosine inverse of the given value; the returned angle is 
in the range
-   * 0.0 through pi.
+   * @return angle in radians whose cosine is `e`, as if computed by 
[[java.lang.Math#acos]]
*
* @group math_funcs
* @since 1.4.0
*/
   def acos(e: Column): Column = withExpr { Acos(e.expr) }
 
+  // scalastyle:off line.size.limit
   /**
-   * Computes the cosine inverse of the given column; the returned angle 
is in the range
-   * 0.0 through pi.
+   * @return angle in radians whose cosine is `columnName` as if computed 
by [[java.lang.Math#acos]]
--- End diff --

It's not a big deal, but this change also makes the way this inverse 
function is described inconsistent with how it's described above. I think it's 
best to make them the same.


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169085966
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
 ---
@@ -770,7 +837,14 @@ case class Unhex(child: Expression) extends 
UnaryExpression with ImplicitCastInp
 
 // scalastyle:off line.size.limit
 @ExpressionDescription(
-  usage = "_FUNC_(expr1, expr2) - Returns the angle in radians between the 
positive x-axis of a plane and the point given by the coordinates (`expr1`, 
`expr2`).",
--- End diff --

I believe this should still be moved with this change.


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169086344
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1313,131 +1313,168 @@ object functions {
   
//
 
   /**
-   * Computes the cosine inverse of the given value; the returned angle is 
in the range
-   * 0.0 through pi.
+   * @return angle in radians whose cosine is `e`, as if computed by 
[[java.lang.Math#acos]]
*
* @group math_funcs
* @since 1.4.0
*/
   def acos(e: Column): Column = withExpr { Acos(e.expr) }
 
+  // scalastyle:off line.size.limit
   /**
-   * Computes the cosine inverse of the given column; the returned angle 
is in the range
-   * 0.0 through pi.
+   * @return angle in radians whose cosine is `columnName` as if computed 
by [[java.lang.Math#acos]]
*
* @group math_funcs
* @since 1.4.0
*/
   def acos(columnName: String): Column = acos(Column(columnName))
 
   /**
-   * Computes the sine inverse of the given value; the returned angle is 
in the range
-   * -pi/2 through pi/2.
+   * @return angle in radians whose sine is `e`, as if computed by 
[[java.lang.Math#asin]]
*
* @group math_funcs
* @since 1.4.0
*/
   def asin(e: Column): Column = withExpr { Asin(e.expr) }
 
   /**
-   * Computes the sine inverse of the given column; the returned angle is 
in the range
-   * -pi/2 through pi/2.
+   * @return angle in radians whose sine is `columnName`, as if computed 
by [[java.lang.Math#asin]]
*
* @group math_funcs
* @since 1.4.0
*/
   def asin(columnName: String): Column = asin(Column(columnName))
 
   /**
-   * Computes the tangent inverse of the given column; the returned angle 
is in the range
-   * -pi/2 through pi/2
+   * @return angle in radians whose tangent is `e`, as if computed by 
[[java.lang.Math#atan]]
*
* @group math_funcs
* @since 1.4.0
*/
   def atan(e: Column): Column = withExpr { Atan(e.expr) }
 
+  // scalastyle:off line.size.limit
   /**
-   * Computes the tangent inverse of the given column; the returned angle 
is in the range
-   * -pi/2 through pi/2
+   * @return angle in radians whose tangent is `columnName`, as if 
computed by [[java.lang.Math#atan]]
*
* @group math_funcs
* @since 1.4.0
*/
   def atan(columnName: String): Column = atan(Column(columnName))
 
   /**
-   * Returns the angle theta from the conversion of rectangular 
coordinates (x, y) to
-   * polar coordinates (r, theta). Units in radians.
+   *
--- End diff --

Tiny nit: remove this first blank line.


---

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



[GitHub] spark pull request #20618: [SPARK-23329][SQL] Fix documentation of trigonome...

2018-02-19 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/20618#discussion_r169086471
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1500,31 +1537,35 @@ object functions {
   }
 
   /**
-   * Computes the cosine of the given value. Units in radians.
+   * @param e angle in radians
+   * @return  cosine of the angle, as if computed by [[java.lang.Math#cos]]
*
* @group math_funcs
* @since 1.4.0
*/
   def cos(e: Column): Column = withExpr { Cos(e.expr) }
 
   /**
-   * Computes the cosine of the given column.
+   * @param columnName angle in radians
+   * @return   cosine of the angle, as if computed by 
[[java.lang.Math#cos]]
--- End diff --

You don't need to line these up, especially if it makes it extend past the 
line limit or wrap.


---

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



  1   2   >