[GitHub] spark pull request #16765: [SPARK-19425][SQL] Make df.except work for UDT

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

https://github.com/apache/spark/pull/16765#discussion_r99066963
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
 ---
@@ -175,6 +175,7 @@ object Literal {
 case map: MapType => create(Map(), map)
 case struct: StructType =>
   create(InternalRow.fromSeq(struct.fields.map(f => 
default(f.dataType).value)), struct)
+case udt: UserDefinedType[_] => default(udt.sqlType)
--- End diff --

Since you are changing the `default` function, could you add a test case to 
`LiteralExpressionSuite`? 


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

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

https://github.com/apache/spark/pull/16775
  
**[Test build #72276 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72276/testReport)**
 for PR 16775 at commit 
[`32c90dd`](https://github.com/apache/spark/commit/32c90dd0817778d3a1a0d1a955463d656dd92d60).


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

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

https://github.com/apache/spark/pull/16766
  
Could you please also add a few test cases?


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

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

https://github.com/apache/spark/pull/16766#discussion_r99065789
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -823,6 +825,17 @@ case class Repartition(numPartitions: Int, shuffle: 
Boolean, child: LogicalPlan)
 }
 
 /**
+  * Returns a new RDD that has exactly `numPartitions` partitions.
+  */
+case class CoalesceLogical(numPartitions: Int, partitionCoalescer: 
Option[PartitionCoalescer],
--- End diff --

`CoalesceLogical ` -> `Coalesce`?


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

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

https://github.com/apache/spark/pull/12135#discussion_r99065474
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
+   * calculation.
+   * @param cols the names of the numerical columns
+   * @param probabilities a list of quantile probabilities
+   *   Each number must belong to [0, 1].
+   *   For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError The relative target precision to achieve (>= 0).
--- End diff --

(BTW, IMHO, at least for now, building javadoc everytime might be good to 
do but not required. We can avoid them at our best in our PRs and then sweep 
them when the release is close or in other related PRs if there are.)


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

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

https://github.com/apache/spark/pull/12135#discussion_r99065162
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
+   * calculation.
+   * @param cols the names of the numerical columns
+   * @param probabilities a list of quantile probabilities
+   *   Each number must belong to [0, 1].
+   *   For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError The relative target precision to achieve (>= 0).
--- End diff --

Maybe, I will ping you if I happened to find another good way to make some 
links for both.


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

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

https://github.com/apache/spark/pull/12135#discussion_r99064944
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
+   * calculation.
+   * @param cols the names of the numerical columns
+   * @param probabilities a list of quantile probabilities
+   *   Each number must belong to [0, 1].
+   *   For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError The relative target precision to achieve (>= 0).
--- End diff --

Yea.. so, kindly @jkbradley opened a JIRA here - 
http://issues.apache.org/jira/browse/SPARK-18692

Actually, they are errors that make documentation building failed in 
javadoc8. I and many guys had a hard time to figure that out a good way AKAIK 
(honestly, I would like to say that I have tried all the combination I could 
think) and it kind of ended up with the one above.. as we are anyway going to 
drop Java 7 support in near future up to my knowledge.


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

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

https://github.com/apache/spark/pull/16766#discussion_r99064595
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -823,6 +825,17 @@ case class Repartition(numPartitions: Int, shuffle: 
Boolean, child: LogicalPlan)
 }
 
 /**
+  * Returns a new RDD that has exactly `numPartitions` partitions.
+  */
+case class CoalesceLogical(numPartitions: Int, partitionCoalescer: 
Option[PartitionCoalescer],
+child: LogicalPlan)
--- End diff --

Could you follow the styles documented in 
https://github.com/databricks/scala-style-guide?


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

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

https://github.com/apache/spark/pull/16664
  
I just quickly went over the code. It looks ok to me, but I will review it 
again when the comments are resolved. Thanks!


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

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



[GitHub] spark pull request #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

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

https://github.com/apache/spark/pull/16664#discussion_r99064088
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -428,8 +481,14 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   partitionColumnNames = partitioningColumns.getOrElse(Nil),
   bucketSpec = getBucketSpec
 )
-df.sparkSession.sessionState.executePlan(
-  CreateTable(tableDesc, mode, Some(df.logicalPlan))).toRdd
+val qe = df.sparkSession.sessionState.executePlan(
+  CreateTable(tableDesc, mode, Some(df.logicalPlan)))
+executeAndCallQEListener(
+  "saveAsTable",
+  qe,
+  new OutputParams(source, Some(tableIdent.unquotedString), 
extraOptions.toMap)) {
--- End diff --

`source`? Why not using a qualified table name?


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

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

https://github.com/apache/spark/pull/16664#discussion_r99063701
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -660,12 +660,21 @@ object SQLConf {
   .booleanConf
   .createWithDefault(false)
 
+
+  val QUERY_EXECUTION_LISTENERS =
--- End diff --

I think we can put it into StaticSQLConf



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

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

https://github.com/apache/spark/pull/16664#discussion_r99063668
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -660,12 +660,21 @@ object SQLConf {
   .booleanConf
   .createWithDefault(false)
 
+
+  val QUERY_EXECUTION_LISTENERS =
+ConfigBuilder("spark.sql.queryExecutionListeners")
+  .doc("QueryExecutionListeners to be attached to the SparkSession")
--- End diff --

I think we can put it into `StaticSQLConf`


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

2017-02-01 Thread lw-lin
Github user lw-lin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16758#discussion_r99063259
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/KeyedState.scala ---
@@ -0,0 +1,134 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.{Experimental, InterfaceStability}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalKeyedState
+
+/**
+ * :: Experimental ::
+ *
+ * Wrapper class for interacting with keyed state data in 
`mapGroupsWithState` and
+ * `flatMapGroupsWithState` operations on
+ * [[KeyValueGroupedDataset]].
+ *
+ * Detail description on `[map/flatMap]GroupsWithState` operation
+ * 
+ * Both, `mapGroupsWithState` and `flatMapGroupsWithState` in 
[[KeyValueGroupedDataset]]
+ * will invoke the user-given function on each group (defined by the 
grouping function in
+ * `Dataset.groupByKey()`) while maintaining user-defined per-group state 
between invocations.
+ * For a static batch Dataset, the function will be invoked once per 
group. For a streaming
+ * Dataset, the function will be invoked for each group repeatedly in 
every trigger.
+ * That is, in every batch of the [[streaming.StreamingQuery 
StreamingQuery]],
+ * the function will be invoked once for each group that has data in the 
batch.
+ *
+ * The function is invoked with following parameters.
+ *  - The key of the group.
+ *  - An iterator containing all the values for this key.
+ *  - A user-defined state object set by previous invocations of the given 
function.
+ * In case of a batch Dataset, there is only invocation and state object 
will be empty as
+ * there is no prior state. Essentially, for batch Datasets, 
`[map/flatMap]GroupsWithState`
+ * is equivalent to `[map/flatMap]Groups`.
+ *
+ * Important points to note about the function.
+ *  - In a trigger, the function will be called only the groups present in 
the batch. So do not
+ *assume that the function will be called in every trigger for every 
group that has state.
+ *  - There is no guaranteed ordering of values in the iterator in the 
function, neither with
+ *batch, nor with streaming Datasets.
+ *  - All the data will be shuffled before applying the function.
+ *
+ * Important points to note about using KeyedState.
+ *  - The value of the state cannot be null. So updating state with null 
is same as removing it.
+ *  - Operations on `KeyedState` are not thread-safe. This is to avoid 
memory barriers.
+ *  - If the `remove()` is called, then `exists()` will return `false`, and
+ *`getOption()` will return `None`.
+ *  - After that `update(newState)` is called, then `exists()` will return 
`true`,
+ *and `getOption()` will return `Some(...)`.
--- End diff --

nit: getOption ...


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

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

https://github.com/apache/spark/pull/16775
  
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 #16775: [SPARK-19433][ML] Periodic checkout datasets for long ml...

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

https://github.com/apache/spark/pull/16775
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72274/
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 #16775: [SPARK-19433][ML] Periodic checkout datasets for long ml...

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

https://github.com/apache/spark/pull/16775
  
**[Test build #72275 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72275/testReport)**
 for PR 16775 at commit 
[`7a1b300`](https://github.com/apache/spark/commit/7a1b3008a5873600016ebe0649285a724c6f4d7c).


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

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

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


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

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



[GitHub] spark pull request #16758: [SPARK-19413][SS] MapGroupsWithState for arbitrar...

2017-02-01 Thread lw-lin
Github user lw-lin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16758#discussion_r99062793
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/MapGroupsWithStateSuite.scala
 ---
@@ -0,0 +1,240 @@
+/*
+ * 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.streaming
+
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.KeyedState
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
+import org.apache.spark.sql.execution.streaming.{KeyedStateImpl, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.StateStore
+
+/** Class to check custom state types */
+case class RunningCount(count: Long)
+
+class MapGroupsWithStateSuite extends StreamTest with BeforeAndAfterAll {
+
+  import testImplicits._
+
+  override def afterAll(): Unit = {
+super.afterAll()
+StateStore.stop()
+  }
+
+  test("state - get, exists, update, remove") {
+var state: KeyedStateImpl[String] = null
+
+def testState(
+expectedData: Option[String],
+shouldBeUpdated: Boolean = false,
+shouldBeRemoved: Boolean = false
+  ): Unit = {
+  if (expectedData.isDefined) {
+assert(state.exists)
+assert(state.get === expectedData.get)
+  } else {
+assert(!state.exists)
+assert(state.get === null)
+  }
+  assert(state.isUpdated === shouldBeUpdated)
+  assert(state.isRemoved === shouldBeRemoved)
+}
+
+// Updating empty state
+state = KeyedStateImpl[String](null)
+testState(None)
+state.update("")
+testState(Some(""), shouldBeUpdated = true)
+
+// Updating exiting state
+state = KeyedStateImpl[String]("2")
+testState(Some("2"))
+state.update("3")
+testState(Some("3"), shouldBeUpdated = true)
+
+// Removing state
+state.remove()
+testState(None, shouldBeRemoved = true, shouldBeUpdated = false)
+state.remove()  // should be still callable
+state.update("4")
+testState(Some("4"), shouldBeRemoved = false, shouldBeUpdated = true)
+
+// Updating by null is same as remove
+state.update(null)
+testState(None, shouldBeRemoved = true, shouldBeUpdated = false)
+  }
+
+  test("flatMapGroupsWithState - streaming") {
+// Function to maintain running count up to 2, and then remove the 
count
+// Returns the data and the count if state is defined, otherwise does 
not return anything
+val stateFunc = (key: String, values: Iterator[String], state: 
KeyedState[RunningCount]) => {
+
+  var count = Option(state.get).map(_.count).getOrElse(0L) + 
values.size
+  if (count == 3) {
+state.remove()
+Iterator.empty
+  } else {
+state.update(RunningCount(count))
+Iterator((key, count.toString))
+  }
+}
+
+val inputData = MemoryStream[String]
+val result =
+  inputData.toDS()
+.groupByKey(x => x)
+.flatMapGroupsWithState(stateFunc) // State: Int, Out: (Str, Str)
+
+testStream(result, Append)(
+  AddData(inputData, "a"),
+  CheckLastBatch(("a", "1")),
+  assertNumStateRows(total = 1, updated = 1),
+  AddData(inputData, "a", "b"),
+  CheckLastBatch(("a", "2"), ("b", "1")),
+  assertNumStateRows(total = 2, updated = 2),
+  StopStream,
+  StartStream(),
+  AddData(inputData, "a", "b"), // should remove state for "a" and not 
return anything for a
+  CheckLastBatch(("b", "2")),
+  assertNumStateRows(total = 1, updated = 2),
+  StopStream,
+  StartStream(),
+  AddData(inputData, "a", "c"), // should recreate state for "a" and 
return count as 1 and
+  

[GitHub] spark pull request #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

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

https://github.com/apache/spark/pull/16664#discussion_r99062729
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -428,8 +481,14 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   partitionColumnNames = partitioningColumns.getOrElse(Nil),
   bucketSpec = getBucketSpec
 )
-df.sparkSession.sessionState.executePlan(
-  CreateTable(tableDesc, mode, Some(df.logicalPlan))).toRdd
+val qe = df.sparkSession.sessionState.executePlan(
+  CreateTable(tableDesc, mode, Some(df.logicalPlan)))
+executeAndCallQEListener(
+  "saveAsTable",
+  qe,
+  new OutputParams(source, Some(tableIdent.unquotedString), 
extraOptions.toMap)) {
+  qe.toRdd
--- End diff --

No need to call `new` here. Please follow the above example. Thanks!


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

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



[GitHub] spark pull request #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

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

https://github.com/apache/spark/pull/16664#discussion_r99062659
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -261,13 +304,19 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   )
 }
 
-df.sparkSession.sessionState.executePlan(
+val qe = df.sparkSession.sessionState.executePlan(
   InsertIntoTable(
 table = UnresolvedRelation(tableIdent),
 partition = Map.empty[String, Option[String]],
 child = df.logicalPlan,
 overwrite = mode == SaveMode.Overwrite,
-ifNotExists = false)).toRdd
+ifNotExists = false))
+executeAndCallQEListener(
+  "insertInto",
+  qe,
+  new OutputParams(source, Some(tableIdent.unquotedString), 
extraOptions.toMap)) {
+qe.toRdd
+}
--- End diff --

Nit: also the style issue.
```Scala
val outputParms = OutputParams(source, Some(tableIdent.unquotedString), 
extraOptions.toMap)
withAction("insertInto", qe, outputParms)(qe.toRdd)
```



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

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

https://github.com/apache/spark/pull/16664#discussion_r99062495
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -218,7 +246,17 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   bucketSpec = getBucketSpec,
   options = extraOptions.toMap)
 
-dataSource.write(mode, df)
+val destination = source match {
+  case "jdbc" => extraOptions.get("dbtable")
+  case _ => extraOptions.get("path")
+}
+
+executeAndCallQEListener(
+  "save",
+  df.queryExecution,
+  OutputParams(source, destination, extraOptions.toMap)) {
+  dataSource.write(mode, df)
+}
--- End diff --

Nit: the style issue.
```Scala
withAction("save", df.queryExecution, OutputParams(source, destination, 
extraOptions.toMap)) {
  dataSource.write(mode, df)
}
```


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

2017-02-01 Thread lw-lin
Github user lw-lin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16758#discussion_r99062438
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/MapGroupsWithStateSuite.scala
 ---
@@ -0,0 +1,240 @@
+/*
+ * 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.streaming
+
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.KeyedState
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
+import org.apache.spark.sql.execution.streaming.{KeyedStateImpl, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.StateStore
+
+/** Class to check custom state types */
+case class RunningCount(count: Long)
+
+class MapGroupsWithStateSuite extends StreamTest with BeforeAndAfterAll {
+
+  import testImplicits._
+
+  override def afterAll(): Unit = {
+super.afterAll()
+StateStore.stop()
+  }
+
+  test("state - get, exists, update, remove") {
+var state: KeyedStateImpl[String] = null
+
+def testState(
+expectedData: Option[String],
+shouldBeUpdated: Boolean = false,
+shouldBeRemoved: Boolean = false
+  ): Unit = {
+  if (expectedData.isDefined) {
+assert(state.exists)
+assert(state.get === expectedData.get)
+  } else {
+assert(!state.exists)
+assert(state.get === null)
+  }
+  assert(state.isUpdated === shouldBeUpdated)
+  assert(state.isRemoved === shouldBeRemoved)
+}
+
+// Updating empty state
+state = KeyedStateImpl[String](null)
+testState(None)
+state.update("")
+testState(Some(""), shouldBeUpdated = true)
+
+// Updating exiting state
+state = KeyedStateImpl[String]("2")
+testState(Some("2"))
+state.update("3")
+testState(Some("3"), shouldBeUpdated = true)
+
+// Removing state
+state.remove()
+testState(None, shouldBeRemoved = true, shouldBeUpdated = false)
+state.remove()  // should be still callable
+state.update("4")
+testState(Some("4"), shouldBeRemoved = false, shouldBeUpdated = true)
+
+// Updating by null is same as remove
+state.update(null)
+testState(None, shouldBeRemoved = true, shouldBeUpdated = false)
+  }
+
+  test("flatMapGroupsWithState - streaming") {
+// Function to maintain running count up to 2, and then remove the 
count
+// Returns the data and the count if state is defined, otherwise does 
not return anything
+val stateFunc = (key: String, values: Iterator[String], state: 
KeyedState[RunningCount]) => {
+
+  var count = Option(state.get).map(_.count).getOrElse(0L) + 
values.size
+  if (count == 3) {
+state.remove()
+Iterator.empty
+  } else {
+state.update(RunningCount(count))
+Iterator((key, count.toString))
+  }
+}
+
+val inputData = MemoryStream[String]
+val result =
+  inputData.toDS()
+.groupByKey(x => x)
+.flatMapGroupsWithState(stateFunc) // State: Int, Out: (Str, Str)
+
+testStream(result, Append)(
+  AddData(inputData, "a"),
+  CheckLastBatch(("a", "1")),
+  assertNumStateRows(total = 1, updated = 1),
+  AddData(inputData, "a", "b"),
+  CheckLastBatch(("a", "2"), ("b", "1")),
+  assertNumStateRows(total = 2, updated = 2),
+  StopStream,
+  StartStream(),
+  AddData(inputData, "a", "b"), // should remove state for "a" and not 
return anything for a
+  CheckLastBatch(("b", "2")),
+  assertNumStateRows(total = 1, updated = 2),
+  StopStream,
+  StartStream(),
+  AddData(inputData, "a", "c"), // should recreate state for "a" and 
return count as 1 and
+  

[GitHub] spark pull request #12135: [SPARK-14352][SQL] approxQuantile should support ...

2017-02-01 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/12135#discussion_r99062470
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
+   * calculation.
+   * @param cols the names of the numerical columns
+   * @param probabilities a list of quantile probabilities
+   *   Each number must belong to [0, 1].
+   *   For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError The relative target precision to achieve (>= 0).
--- End diff --

Are these just warnings generated? It would be nice to know during Jenkins 
testing if javadoc8 (or scaladoc for that matter) breaks.

The 2nd case links nicely to the single-arg version of the method, which 
contains the detailed doc, in Scaladoc. Pity it won't work with javadoc - is 
there another way to link it correctly? I suspect that what will work for 
javadoc will break the link for scaladoc...


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

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

https://github.com/apache/spark/pull/16664#discussion_r99062185
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -514,6 +576,9 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
* shorten names(none, `snappy`, `gzip`, and `lzo`). This will override
* `spark.sql.parquet.compression.codec`.
* 
+   * Calls the callback methods in @see[[QueryExecutionListener]] methods 
after query execution with
+   * @see[[OutputParams]] having datasourceType set as string constant 
"parquet" and
+   * destination set as the path to which the data is written
--- End diff --

I think we do not need to add these comments to all the functions.


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

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

https://github.com/apache/spark/pull/16664#discussion_r99062037
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -660,12 +660,21 @@ object SQLConf {
   .booleanConf
   .createWithDefault(false)
 
+
+  val QUERY_EXECUTION_LISTENERS =
+ConfigBuilder("spark.sql.queryExecutionListeners")
+  .doc("QueryExecutionListeners to be attached to the SparkSession")
--- End diff --

Can you improve this line? Add what you wrote in the 
`sql-programming-guide.md`?


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

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

https://github.com/apache/spark/pull/16664#discussion_r99061828
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -190,6 +192,32 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   }
 
   /**
+   * Executes the query and calls the {@link 
org.apache.spark.sql.util.QueryExecutionListener}
+   * methods.
+   *
+   * @param funcName A identifier for the method executing the query
+   * @param qe the @see [[QueryExecution]] object associated with the query
+   * @param outputParams The output parameters useful for query analysis
+   * @param action the function that executes the query after which the 
listener methods gets
+   *   called.
+   */
+  private def executeAndCallQEListener(
--- End diff --

How about renaming it `withAction`? It is more consistent.


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

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

https://github.com/apache/spark/pull/16664#discussion_r99061710
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -190,6 +192,32 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   }
 
   /**
+   * Executes the query and calls the {@link 
org.apache.spark.sql.util.QueryExecutionListener}
+   * methods.
--- End diff --

How about changing it to
> > Wrap a DataFrameWriter action to track the QueryExecution and time 
cost, then report to the user-registered callback functions.




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

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

https://github.com/apache/spark/pull/16664
  
@marmbrus `DataStreamWriter` has similar issues, right?


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

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

https://github.com/apache/spark/pull/16664#discussion_r99060951
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -190,6 +192,32 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   }
 
   /**
+   * Executes the query and calls the {@link 
org.apache.spark.sql.util.QueryExecutionListener}
+   * methods.
+   *
+   * @param funcName A identifier for the method executing the query
+   * @param qe the @see [[QueryExecution]] object associated with the query
--- End diff --

Could you please fix the doc by following what 
https://github.com/apache/spark/pull/16013 did? 


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

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

https://github.com/apache/spark/pull/16664#discussion_r99060523
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -660,12 +660,21 @@ object SQLConf {
   .booleanConf
   .createWithDefault(false)
 
+
+  val QUERY_EXECUTION_LISTENERS =
+ConfigBuilder("spark.sql.queryExecutionListeners")
+  .doc("QueryExecutionListeners to be attached to the SparkSession")
+  .stringConf
+  .toSequence
+  .createWithDefault(Nil)
+
   val SESSION_LOCAL_TIMEZONE =
 SQLConfigBuilder("spark.sql.session.timeZone")
   .doc("""The ID of session local timezone, e.g. "GMT", 
"America/Los_Angeles", etc.""")
   .stringConf
   .createWithDefault(TimeZone.getDefault().getID())
 
+
--- End diff --

Nit: Please remove this empty line


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

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

https://github.com/apache/spark/pull/12135
  
@zhengruifeng Please try to improve the test case coverage in the follow-up 
PRs. You might find some bugs when you added these test cases. Thanks for your 
work!


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

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

https://github.com/apache/spark/pull/12135#discussion_r99060083
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
+   * calculation.
+   * @param cols the names of the numerical columns
+   * @param probabilities a list of quantile probabilities
+   *   Each number must belong to [0, 1].
--- End diff --

What happened if the users provide the number that is not in this boundary? 
Do we have a test case to verify the behavior?


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

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

https://github.com/apache/spark/pull/12135#discussion_r99059985
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
+   * calculation.
+   * @param cols the names of the numerical columns
+   * @param probabilities a list of quantile probabilities
+   *   Each number must belong to [0, 1].
+   *   For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError The relative target precision to achieve (>= 0).
+   *   If set to zero, the exact quantiles are computed, which could be 
very expensive.
--- End diff --

This case is also missing. 

Actually, you also need to consider the illegal cases, like negative values.


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

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

https://github.com/apache/spark/pull/12135#discussion_r99059884
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
+   * calculation.
+   * @param cols the names of the numerical columns
+   * @param probabilities a list of quantile probabilities
+   *   Each number must belong to [0, 1].
+   *   For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError The relative target precision to achieve (>= 0).
+   *   If set to zero, the exact quantiles are computed, which could be 
very expensive.
+   *   Note that values greater than 1 are accepted but give the same 
result as 1.
--- End diff --

It sounds like you did not add any test case to verify it. 


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

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



[GitHub] spark pull request #16758: [SPARK-19413][SS] MapGroupsWithState for arbitrar...

2017-02-01 Thread lw-lin
Github user lw-lin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16758#discussion_r99059679
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/KeyedStateImpl.scala
 ---
@@ -0,0 +1,57 @@
+/*
+ * 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.streaming
+
+import org.apache.spark.sql.KeyedState
+
+/** Internal implementation of the [[KeyedState]] interface */
+private[sql] case class KeyedStateImpl[S](private var value: S) extends 
KeyedState[S] {
+  private var updated: Boolean = false  // whether value has been updated 
(but not removed)
+  private var removed: Boolean = false  // whether value has been removed
+
+  // = Public API =
+  override def exists: Boolean = { value != null }
+
+  override def get: S = value
+
+  override def update(newValue: S): Unit = {
+if (newValue == null) {
+  remove()
+} else {
+  value = newValue
+  updated = true
+  removed = false
+}
+  }
+
+  override def remove(): Unit = {
+value = null.asInstanceOf[S]
+updated = false
+removed = true
+  }
+
+  override def toString: String = "KeyedState($value)"
--- End diff --

nit: _s_"KeyedState($value)"


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

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

https://github.com/apache/spark/pull/12135
  
@zhengruifeng Actually, I still have a few comments about this PR. I will 
leave the comments soon. Thanks!


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

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



[GitHub] spark pull request #12135: [SPARK-14352][SQL] approxQuantile should support ...

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

https://github.com/apache/spark/pull/12135#discussion_r99059030
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
--- End diff --

`values values` -> `values` 

@zhengruifeng Could you submit a follow-up PR to add test cases for null 
values?


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

2017-02-01 Thread zhengruifeng
Github user zhengruifeng commented on the issue:

https://github.com/apache/spark/pull/12135
  
@HyukjinKwon @gatorsmile Thanks for pointing out those issues. I will 
create a followup PR to fix them ASAP.


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

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

https://github.com/apache/spark/pull/12135
  
@holdenk When you do the code merge, you need to leave a comment to explain 
which branch you merged. 


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

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

https://github.com/apache/spark/pull/12135#discussion_r99057948
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
+   * calculation.
+   * @param cols the names of the numerical columns
+   * @param probabilities a list of quantile probabilities
+   *   Each number must belong to [0, 1].
+   *   For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError The relative target precision to achieve (>= 0).
+   *   If set to zero, the exact quantiles are computed, which could be 
very expensive.
+   *   Note that values greater than 1 are accepted but give the same 
result as 1.
+   * @return the approximate quantiles at the given probabilities of each 
column
+   *
+   * @note Rows containing any NaN values will be removed before 
calculation
+   *
+   * @since 2.2.0
+   */
+  def approxQuantile(
+  cols: Array[String],
+  probabilities: Array[Double],
+  relativeError: Double): Array[Array[Double]] = {
+StatFunctions.multipleApproxQuantiles(df.select(cols.map(col): 
_*).na.drop(), cols,
+  probabilities, relativeError).map(_.toArray).toArray
+  }
+
+
+  /**
* Python-friendly version of [[approxQuantile()]]
*/
   private[spark] def approxQuantile(
-  col: String,
+  cols: List[String],
   probabilities: List[Double],
-  relativeError: Double): java.util.List[Double] = {
-approxQuantile(col, probabilities.toArray, relativeError).toList.asJava
+  relativeError: Double): java.util.List[java.util.List[Double]] = {
+approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+.map(_.toList.asJava).toList.asJava
--- End diff --

The indent is not right.


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

2017-02-01 Thread thomastechs
Github user thomastechs commented on the issue:

https://github.com/apache/spark/pull/16743
  
One point, as discussed, statusChange gets called for task status change. 
So, if we can identify the point where the job or that executor(Only one 
executor for local mode, right) is failed, we can give  a call back at that 
point. If you could give some insights about it, that fix can be applied.


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

2017-02-01 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/16761
  
hmm, I wasn't sure to have the parameter changes in 2.1, what do you think?



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

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



[GitHub] spark pull request #16722: [SPARK-9478][ML][MLlib] Add sample weights to dec...

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99056238
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
@@ -23,6 +23,7 @@ import scala.util.Try
 import org.apache.spark.annotation.Since
 import org.apache.spark.api.java.JavaRDD
 import org.apache.spark.internal.Logging
+import org.apache.spark.ml.feature.{Instance => NewInstance}
--- End diff --

the mllib code is referencing ml code... scary and weird, but I guess not 
much you can do here, just seems really convoluted


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16689: [SPARK-19342][SPARKR] bug fixed in collect method for co...

2017-02-01 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/16689
  
hmm, that's not a super big issue since vector and list is more or less the 
same in R.
I think it might be better if we are treating the type consistently, 
although it might be a concerning if this is changing in a non-backward 
compatible manner.

let me try to find some time to test this out? thanks!


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

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



[GitHub] spark pull request #16722: [SPARK-9478][ML][MLlib] Add sample weights to dec...

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99056047
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala ---
@@ -72,6 +72,21 @@ private[ml] trait DecisionTreeParams extends 
PredictorParams
 " Should be >= 1.", ParamValidators.gtEq(1))
 
   /**
+   * Minimum fraction of the weighted sample count that each child must 
have after split.
+   * If a split causes the fraction of the total weight in the left or 
right child to be less than
+   * minWeightFractionPerNode, the split will be discarded as invalid.
+   * Should be in the interval [0.0, 0.5).
+   * (default = 0.0)
+   * @group param
+   */
+  final val minWeightFractionPerNode: DoubleParam = new DoubleParam(this,
+"minWeightFractionPerNode", "Minimum fraction of the weighted sample 
count that each child " +
+"must have after split. If a split causes the fraction of the total 
weight in the left or " +
+"or right child to be less than minWeightFractionPerNode, the split 
will be discarded as " +
--- End diff --

minor: two "or"s here, remove one


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

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99055889
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala ---
@@ -590,8 +599,8 @@ private[spark] object RandomForest extends Logging {
 if (!isLeaf) {
   node.split = Some(split)
   val childIsLeaf = (LearningNode.indexToLevel(nodeIndex) + 1) == 
metadata.maxDepth
-  val leftChildIsLeaf = childIsLeaf || (stats.leftImpurity == 0.0)
-  val rightChildIsLeaf = childIsLeaf || (stats.rightImpurity == 
0.0)
+  val leftChildIsLeaf = childIsLeaf || 
(math.abs(stats.leftImpurity) < 1e-16)
+  val rightChildIsLeaf = childIsLeaf || 
(math.abs(stats.rightImpurity) < 1e-16)
--- End diff --

the code for left/right child looks very similar, consider refactoring to a 
function.
also, should 1e-16 be moved to a constant, or is there a global constant 
somewhere for this (or how was this value chosen)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16767: [SPARK-19386][SPARKR][DOC] Bisecting k-means in S...

2017-02-01 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/16767#discussion_r99055560
  
--- Diff: R/pkg/vignettes/sparkr-vignettes.Rmd ---
@@ -819,6 +821,18 @@ perplexity <- spark.perplexity(model, corpusDF)
 perplexity
 ```
 
+ Bisecting k-means
--- End diff --

same here. the model sections are in alphabetic order


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

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16767: [SPARK-19386][SPARKR][DOC] Bisecting k-means in S...

2017-02-01 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/16767#discussion_r99055524
  
--- Diff: R/pkg/vignettes/sparkr-vignettes.Rmd ---
@@ -494,6 +494,8 @@ SparkR supports the following machine learning models 
and algorithms.
 
 * Latent Dirichlet Allocation (LDA)
 
+* Bisecting $k$-means
--- End diff --

these model names are in order
```
 Clustering
 
 * Gaussian Mixture Model (GMM)
 
 * $k$-means Clustering
  
  * Latent Dirichlet Allocation (LDA)
  
 * Bisecting $k$-means
```

should be
```
 Clustering
 
 * Bisecting $k$-means

 * Gaussian Mixture Model (GMM)
 
 * $k$-means Clustering
  
  * Latent Dirichlet Allocation (LDA)
  
 ```


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

2017-02-01 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/16729
  
ah thanks. so I were to 
```
library(statmod)
library(SparkR)
```

could I still access the statmod tweedie function?
ie. does statmod::tweedie still work with R base::glm?


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

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99055341
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala 
---
@@ -42,6 +42,7 @@ import org.apache.spark.rdd.RDD
 private[spark] class DecisionTreeMetadata(
 val numFeatures: Int,
 val numExamples: Long,
+val weightedNumExamples: Double,
--- End diff --

shouldn't the new params weightedNumExamples and  minWeightFractionPerNode 
be added to the documentation for this 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 issue #16773: [SPARK-19432][Core]Fix an unexpected failure when connec...

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

https://github.com/apache/spark/pull/16773
  
Thanks. Merging to master and 2.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 issue #16739: [SPARK-19399][SPARKR] Add R coalesce API for DataFrame a...

2017-02-01 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/16739
  
yap, https://github.com/apache/spark/pull/16739#issuecomment-276739220 - 
only RDD has `coalesce(.. shuffle)`, in Dataset, it's `coalesce` and 
`repartition`


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

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99054611
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala 
---
@@ -117,20 +114,20 @@ class RandomForestRegressor @Since("1.4.0") 
(@Since("1.4.0") override val uid: S
   override protected def train(dataset: Dataset[_]): 
RandomForestRegressionModel = {
 val categoricalFeatures: Map[Int, Int] =
   MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol)))
-val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset)
+
+val instances = extractLabeledPoints(dataset).map(_.toInstance(1.0))
--- End diff --

simplify to toInstance (without the 1.0)


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

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



[GitHub] spark pull request #16722: [SPARK-9478][ML][MLlib] Add sample weights to dec...

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99054576
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala 
---
@@ -99,16 +105,31 @@ class DecisionTreeRegressor @Since("1.4.0") 
(@Since("1.4.0") override val uid: S
   @Since("2.0.0")
   def setVarianceCol(value: String): this.type = set(varianceCol, value)
 
+  /**
+   * Sets the value of param [[weightCol]].
+   * If this is not set or empty, we treat all instance weights as 1.0.
+   * Default is not set, so all instances have weight one.
+   *
+   * @group setParam
+   */
+  @Since("2.2.0")
+  def setWeightCol(value: String): this.type = set(weightCol, value)
+
   override protected def train(dataset: Dataset[_]): 
DecisionTreeRegressionModel = {
 val categoricalFeatures: Map[Int, Int] =
   MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol)))
-val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset)
+val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) 
else col($(weightCol))
+val instances =
+  dataset.select(col($(labelCol)).cast(DoubleType), w, 
col($(featuresCol))).rdd.map {
+case Row(label: Double, weight: Double, features: Vector) =>
+  Instance(label, weight, features)
+  }
--- End diff --

the code above looks the same as the classifier, can we refactor somehow:

val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) 
else col($(weightCol)) 
val instances = 
  dataset.select(col($(labelCol)).cast(DoubleType), w, 
col($(featuresCol))).rdd.map { 
case Row(label: Double, weight: Double, features: Vector) => 
  Instance(label, weight, features) 



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

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99054369
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala
 ---
@@ -126,20 +127,20 @@ class RandomForestClassifier @Since("1.4.0") (
 s" numClasses=$numClasses, but thresholds has length 
${$(thresholds).length}")
 }
 
-val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, 
numClasses)
+val instances: RDD[Instance] = extractLabeledPoints(dataset, 
numClasses).map(_.toInstance(1.0))
--- End diff --

it looks like this:
toInstance(1.0)
can just be simplified as:
toInstance


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

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

https://github.com/apache/spark/pull/16729
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72273/
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 #16722: [SPARK-9478][ML][MLlib] Add sample weights to dec...

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99054331
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala ---
@@ -35,4 +35,11 @@ case class LabeledPoint(@Since("2.0.0") label: Double, 
@Since("2.0.0") features:
   override def toString: String = {
 s"($label,$features)"
   }
+
+  private[spark] def toInstance: Instance = toInstance(1.0)
--- End diff --

this is kind of a nit pick, and optional, but I would usually refactor out 
magic numbers like 1.0 as something like "defaultWeight" and reuse it 
elsewhere, but it's not really necessary in this case since it probably won't 
ever change


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

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



[GitHub] spark issue #16729: [SPARK-19391][SparkR][ML] Tweedie GLM API for SparkR

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

https://github.com/apache/spark/pull/16729
  
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 #16729: [SPARK-19391][SparkR][ML] Tweedie GLM API for SparkR

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

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


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

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



[GitHub] spark pull request #16722: [SPARK-9478][ML][MLlib] Add sample weights to dec...

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99054115
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala
 ---
@@ -106,14 +122,18 @@ class DecisionTreeClassifier @Since("1.4.0") (
 ".train() called with non-matching numClasses and 
thresholds.length." +
 s" numClasses=$numClasses, but thresholds has length 
${$(thresholds).length}")
 }
-
-val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, 
numClasses)
--- End diff --

I would say that's fine if it was only in one place, but I also see this 
pattern in DecisionTreeRegressor.scala, it seems like we should be able to 
refactor this part out


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

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

https://github.com/apache/spark/pull/16775
  
also cc @MLnick 


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

2017-02-01 Thread imatiach-msft
Github user imatiach-msft commented on a diff in the pull request:

https://github.com/apache/spark/pull/16722#discussion_r99053832
  
--- Diff: 
mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala ---
@@ -48,7 +48,7 @@ object TestingUtils {
   /**
* Private helper function for comparing two values using absolute 
tolerance.
*/
-  private def AbsoluteErrorComparison(x: Double, y: Double, eps: Double): 
Boolean = {
+  private[ml] def AbsoluteErrorComparison(x: Double, y: Double, eps: 
Double): Boolean = {
--- End diff --

ok, I don't have a very strong opinion here either


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

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

https://github.com/apache/spark/pull/16775
  
cc @mengxr @jkbradley @liancheng  


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

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

https://github.com/apache/spark/pull/16775
  
**[Test build #72274 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72274/testReport)**
 for PR 16775 at commit 
[`5ed5c2a`](https://github.com/apache/spark/commit/5ed5c2a65c31c78b7845bbb8a3ef859590453ba9).


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

2017-02-01 Thread viirya
GitHub user viirya opened a pull request:

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

[WIP][ML] Periodic checkout datasets for long ml pipeline

## What changes were proposed in this pull request?

WIP

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a 
pull request.


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

$ git pull https://github.com/viirya/spark-1 
periodic-checkout-for-long-ml-pipeline

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

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


commit 5ed5c2a65c31c78b7845bbb8a3ef859590453ba9
Author: Liang-Chi Hsieh 
Date:   2017-02-02T04:49:04Z

Periodic checkout datasets for long ml pipeline.




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

2017-02-01 Thread drcrallen
Github user drcrallen commented on the issue:

https://github.com/apache/spark/pull/16714
  
@vanzin can you check this out 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 #16729: [SPARK-19391][SparkR][ML] Tweedie GLM API for SparkR

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

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


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

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

https://github.com/apache/spark/pull/16765
  
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 #16765: [SPARK-19425][SQL] Make df.except work for UDT

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

https://github.com/apache/spark/pull/16765
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72272/
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 #16765: [SPARK-19425][SQL] Make df.except work for UDT

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

https://github.com/apache/spark/pull/16765
  
**[Test build #72272 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72272/testReport)**
 for PR 16765 at commit 
[`af98964`](https://github.com/apache/spark/commit/af9896466313a69ab76b38e46aeb48abad28f74c).
 * 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 #16690: [SPARK-19347] ReceiverSupervisorImpl can add block to Re...

2017-02-01 Thread jinxing64
Github user jinxing64 commented on the issue:

https://github.com/apache/spark/pull/16690
  
Thanks a lot for reviewing this PR~


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

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



[GitHub] spark issue #16773: [SPARK-19432][Core]Fix an unexpected failure when connec...

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

https://github.com/apache/spark/pull/16773
  
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 #16773: [SPARK-19432][Core]Fix an unexpected failure when connec...

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

https://github.com/apache/spark/pull/16773
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72266/
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 #16773: [SPARK-19432][Core]Fix an unexpected failure when connec...

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

https://github.com/apache/spark/pull/16773
  
**[Test build #72266 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72266/testReport)**
 for PR 16773 at commit 
[`ee695a8`](https://github.com/apache/spark/commit/ee695a84866c7d099756cc023a782ec0749a0ce5).
 * 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 #16689: [SPARK-19342][SPARKR] bug fixed in collect method for co...

2017-02-01 Thread titicaca
Github user titicaca commented on the issue:

https://github.com/apache/spark/pull/16689
  
I tried to modify the PRIMITIVE_TYPES for timestamp, but it had a side 
effect on coltypes method.

In test_sparkSQL.R#2262, `expect_equal(coltypes(DF), c("integer", 
"logical", "POSIXct"))`, coltypes return a list instead of a vector because of 
the convertion from timestamp to `c(POSIXct, POSIXt)`


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

2017-02-01 Thread adamstatdna
Github user adamstatdna commented on the issue:

https://github.com/apache/spark/pull/16743
  
My use case is end-to-end automated testing in local mode using 
programmatic Launcher. I have tests where the Spark app is expected to be 
FINISHED and those where it is expected to be 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 #16763: [SPARK-19422][ML] Cache input data in algorithms

2017-02-01 Thread zhengruifeng
Github user zhengruifeng commented on the issue:

https://github.com/apache/spark/pull/16763
  
@hhbyyh Thanks a lot for pointing this out!


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

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

https://github.com/apache/spark/pull/12135#discussion_r99040876
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala ---
@@ -75,13 +76,43 @@ final class DataFrameStatFunctions private[sql](df: 
DataFrame) {
   }
 
   /**
+   * Calculates the approximate quantiles of numerical columns of a 
DataFrame.
+   * @see [[DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile]] for
+   * detailed description.
+   *
+   * Note that rows containing any null or NaN values values will be 
removed before
+   * calculation.
+   * @param cols the names of the numerical columns
+   * @param probabilities a list of quantile probabilities
+   *   Each number must belong to [0, 1].
+   *   For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError The relative target precision to achieve (>= 0).
--- End diff --

As a kind comment to inform as I know it is super easy for javadoc8 to be 
broken It seems javadoc8 complains it as below:

```
[error] 
.../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:43:
 error: unexpected content
[error]* @see {@link DataFrameStatsFunctions.approxQuantile(col:Str* 
approxQuantile} for
[error]  ^
[error] 
.../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:52:
 error: bad use of '>'
[error]* @param relativeError The relative target precision to achieve 
(>= 0).
[error]
```

We could do this as

```
@param relativeError The relative target precision to achieve (greater or 
equal to 0).
```

and fix the link as below _If there is no better choice_:

```
@see `DataFrameStatsFunctions.approxQuantile` for detailed description.
```

Just FYI, there are several cases in 
https://github.com/apache/spark/pull/16013


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

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

https://github.com/apache/spark/pull/16772
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72271/
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 #16772: [SPARK-14772][PYTHON][ML] Fixed Params.copy method to ma...

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

https://github.com/apache/spark/pull/16772
  
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 #16772: [SPARK-14772][PYTHON][ML] Fixed Params.copy method to ma...

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

https://github.com/apache/spark/pull/16772
  
**[Test build #72271 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72271/testReport)**
 for PR 16772 at commit 
[`ce59d74`](https://github.com/apache/spark/commit/ce59d745c5bd5f2674822e1f937face5b2e509f6).
 * 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 #12420: [SPARK-14585][ML][WIP] Provide accessor methods for Pipe...

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

https://github.com/apache/spark/pull/12420
  
I missed the ClassTag question above.  Let me take a look


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

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

https://github.com/apache/spark/pull/16729
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72268/
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 #16729: [SPARK-19391][SparkR][ML] Tweedie GLM API for SparkR

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

https://github.com/apache/spark/pull/16729
  
**[Test build #72268 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72268/testReport)**
 for PR 16729 at commit 
[`b10777e`](https://github.com/apache/spark/commit/b10777eb08621141df7190daa6157ff064c9d1af).
 * This patch **fails SparkR 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 #16729: [SPARK-19391][SparkR][ML] Tweedie GLM API for SparkR

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

https://github.com/apache/spark/pull/16729
  
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 #16765: [SPARK-19425][SQL] Make df.except work for UDT

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

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


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

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

https://github.com/apache/spark/pull/16765
  
Simplified the code change.


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

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



[GitHub] spark issue #16723: [SPARK-19389][ML][PYTHON][DOC] Minor doc fixes for ML Py...

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

https://github.com/apache/spark/pull/16723
  
(I just rebased it based on this PR and built the javadoc8 for sure. I 
believe it should emit an error if this PR introduce the break but it seems 
not. So, LGTM for doc changes.)


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

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

https://github.com/apache/spark/pull/16737
  
Please hold on this PR. Found a serious bug to fix in case insensitive 
option support.


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

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

https://github.com/apache/spark/pull/16774
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72267/
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 #16774: [SPARK-19357][ML][WIP] Adding parallel model evaluation ...

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

https://github.com/apache/spark/pull/16774
  
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 #16774: [SPARK-19357][ML][WIP] Adding parallel model evaluation ...

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

https://github.com/apache/spark/pull/16774
  
**[Test build #72267 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72267/testReport)**
 for PR 16774 at commit 
[`5650e98`](https://github.com/apache/spark/commit/5650e98a580544303dc1185568be992d9304707a).
 * 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 #16772: [SPARK-14772][PYTHON][ML] Fixed Params.copy method to ma...

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

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


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

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

https://github.com/apache/spark/pull/16771
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72269/
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 #16771: [SPARK-19429][PYTHON][SQL] Support slice arguments in Co...

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

https://github.com/apache/spark/pull/16771
  
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 #16771: [SPARK-19429][PYTHON][SQL] Support slice arguments in Co...

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

https://github.com/apache/spark/pull/16771
  
**[Test build #72269 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72269/testReport)**
 for PR 16771 at commit 
[`c1f5110`](https://github.com/apache/spark/commit/c1f5110ee173320dc6d6d14146752c8517858271).
 * 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 #16772: [SPARK-14772][PYTHON][ML] Fixed Params.copy method to ma...

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

https://github.com/apache/spark/pull/16772
  
**[Test build #72270 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72270/testReport)**
 for PR 16772 at commit 
[`c25c127`](https://github.com/apache/spark/commit/c25c127dd89f871ac57f8f62b080e33db4ab9f2b).
 * This patch **fails PySpark 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 #16772: [SPARK-14772][PYTHON][ML] Fixed Params.copy method to ma...

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

https://github.com/apache/spark/pull/16772
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72270/
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 #16772: [SPARK-14772][PYTHON][ML] Fixed Params.copy method to ma...

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

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



  1   2   3   4   >