[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-10-15 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-148306575
  
we can create a hash expression, and codegen that. And then just use 
hyperloglog(hash(field)).


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

2015-10-15 Thread hvanhovell
Github user hvanhovell commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-148305817
  
Another thought on hashing. The ClearSpring hash is a generic hash 
function. We could used very specialized (hopefully fast) hashing functions, 
because we know the type of our input.


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

2015-10-14 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/8362#discussion_r42054243
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
 ---
@@ -302,3 +307,397 @@ case class Sum(child: Expression) extends 
AlgebraicAggregate {
 
   override val evaluateExpression = Cast(currentSum, resultType)
 }
+
+// scalastyle:off
+/**
+ * HyperLogLog++ (HLL++) is a state of the art cardinality estimation 
algorithm. This class
+ * implements the dense version of the HLL++ algorithm as an Aggregate 
Function.
+ *
+ * This implementation has been based on the following papers:
+ * HyperLogLog: the analysis of a near-optimal cardinality estimation 
algorithm
+ * http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf
+ *
+ * HyperLogLog in Practice: Algorithmic Engineering of a State of The Art 
Cardinality Estimation
+ * Algorithm
+ * 
http://static.googleusercontent.com/external_content/untrusted_dlcp/research.google.com/en/us/pubs/archive/40671.pdf
+ *
+ * Appendix to HyperLogLog in Practice: Algorithmic Engineering of a State 
of the Art Cardinality
+ * Estimation Algorithm
+ * 
https://docs.google.com/document/d/1gyjfMHy43U9OWBXxfaeG-3MjGzejW1dlpyMwEYAAWEI/view?fullscreen#
+ *
+ * @param child to estimate the cardinality of.
+ * @param relativeSD the maximum estimation error allowed.
+ */
+// scalastyle:on
+case class HyperLogLogPlusPlus(child: Expression, relativeSD: Double = 
0.05)
+extends AggregateFunction2 {
+  import HyperLogLogPlusPlus._
+
+  /**
+   * HLL++ uses 'p' bits for addressing. The more addressing bits we use, 
the more precise the
+   * algorithm will be, and the more memory it will require. The 'p' value 
is based on the relative
+   * error requested.
+   *
+   * HLL++ requires that we use at least 4 bits of addressing space (a 
minimum precision of 27%).
+   *
+   * This method rounds up to the nearest integer. This means that the 
error is always equal to or
+   * lower than the requested error. Use the trueRsd method 
to get the actual RSD
+   * value.
+   */
+  private[this] val p = Math.ceil(2.0d * Math.log(1.106d / relativeSD) / 
Math.log(2.0d)).toInt
+
+  require(p >= 4, "HLL++ requires at least 4 bits for addressing. " +
+"Use a lower error, at most 27%.")
+
+  /**
+   * Shift used to extract the index of the register from the hashed value.
+   *
+   * This assumes the use of 64-bit hashcodes.
+   */
+  private[this] val idxShift = JLong.SIZE - p
+
+  /**
+   * Value to pad the 'w' value with before the number of leading zeros is 
determined.
+   */
+  private[this] val wPadding = 1L << (p - 1)
+
+  /**
+   * The number of registers used.
+   */
+  private[this] val m = 1 << p
+
+  /**
+   * The pre-calculated combination of: alpha * m * m
+   *
+   * 'alpha' corrects the raw cardinality estimate 'Z'. See the FlFuGaMe07 
paper for its
+   * derivation.
+   */
+  private[this] val alphaM2 = p match {
+case 4 => 0.673d * m * m
+case 5 => 0.697d * m * m
+case 6 => 0.709d * m * m
+case _ => (0.7213d / (1.0d + 1.079d / m)) * m * m
+  }
+
+  /**
+   * The number of words used to store the registers. We use Longs for 
storage because this is the
+   * most compact way of storage; Spark aligns to 8-byte words or uses 
Long wrappers.
+   *
+   * We only store whole registers per word in order to prevent overly 
complex bitwise operations.
+   * In practice this means we only use 60 out of 64 bits.
+   */
+  private[this] val numWords = m / REGISTERS_PER_WORD + 1
+
+  def children: Seq[Expression] = Seq(child)
+
+  def nullable: Boolean = false
+
+  def dataType: DataType = LongType
+
+  def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)
+
+  def bufferSchema: StructType = 
StructType.fromAttributes(bufferAttributes)
+
+  def cloneBufferAttributes: Seq[Attribute] = 
bufferAttributes.map(_.newInstance())
+
+  /** Allocate enough words to store all registers. */
+  val bufferAttributes: Seq[AttributeReference] = Seq.tabulate(numWords) { 
i =>
+AttributeReference(s"MS[$i]", LongType)()
+  }
+
+  /** Fill all words with zeros. */
+  def initialize(buffer: MutableRow): Unit = {
+var word = 0
+while (word < numWords) {
+  buffer.setLong(mutableBufferOffset + word, 0)
+  word += 1
+}
+  }
+
+  /**
+   * Update the HLL++ buffer.
+   *
+   * Variable names in the HLL++ paper match variable names in the code.
+   */
+  def update(buffer: MutableRow, input: InternalRow): Unit = {
+val 

[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-10-14 Thread hvanhovell
Github user hvanhovell commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-148209375
  
@yhuai It doesn't. A 64-bit hashcode is recommended though, especially when 
would want to approximate a billion or more unique values. I have used the 
ClearSpring hashcode, because this enabled me to compare the results of my 
HLL++ implementation to theirs.

We could replace it with another, better performing, one; don't we have one 
in Spark? We could also scale down to 32-bits...


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

2015-10-14 Thread hvanhovell
Github user hvanhovell commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-148209543
  
A good article on HLL++ and the hashcode: 
http://research.neustar.biz/2013/01/24/hyperloglog-googles-take-on-engineering-hll


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

2015-10-14 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-148227211
  
Thanks for the pointer. Looks like we only have 32-bit Murmur3 in spark's 
unsafe module 
(https://github.com/apache/spark/blob/master/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java).
 


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

2015-09-30 Thread asfgit
Github user asfgit closed the pull request at:

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


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

2015-09-30 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/8362#discussion_r40782166
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala
 ---
@@ -0,0 +1,125 @@
+/*
+ * 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.catalyst.expressions.aggregate
+
+import java.util.Random
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{MutableRow, 
BoundReference, SpecificMutableRow}
+import org.apache.spark.sql.types.IntegerType
+
+import scala.collection.mutable
+import org.scalatest.Assertions._
+
+/**
+ * Created by hvanhovell on 8/27/15.
--- End diff --

I'll remove 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: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-144390907
  
Merged build started.


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

2015-09-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-144390880
  
 Merged build triggered.


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

2015-09-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-144391587
  
  [Test build #43132 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43132/consoleFull)
 for   PR 8362 at commit 
[`a5fdd07`](https://github.com/apache/spark/commit/a5fdd07005547ee3c93db1f8a1da7327d6ca1c27).


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

2015-09-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-11434
  
  [Test build #43132 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43132/console)
 for   PR 8362 at commit 
[`a5fdd07`](https://github.com/apache/spark/commit/a5fdd07005547ee3c93db1f8a1da7327d6ca1c27).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class HyperLogLogPlusPlus(child: Expression, relativeSD: Double = 
0.05)`



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

2015-09-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-11597
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43132/
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: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-11593
  
Merged build finished. Test PASSed.


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

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



[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-30 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-144479889
  
mer


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

2015-09-30 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-144479867
  
LGTM, merging this into master, 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: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-29 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-144118463
  
@hvanhovell @rxin Just realized that the tungsten aggregation does not 
support var-length types in aggregation buffer, so we can't have sparse version 
without aggregation 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 pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-29 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/8362#discussion_r40697020
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala
 ---
@@ -0,0 +1,125 @@
+/*
+ * 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.catalyst.expressions.aggregate
+
+import java.util.Random
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{MutableRow, 
BoundReference, SpecificMutableRow}
+import org.apache.spark.sql.types.IntegerType
+
+import scala.collection.mutable
+import org.scalatest.Assertions._
+
+/**
+ * Created by hvanhovell on 8/27/15.
--- End diff --

We usually didn't have this kind of comments (generated by IntellJ), could 
you update it or remove 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: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-29 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/8362#discussion_r40696819
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
 ---
@@ -302,3 +307,393 @@ case class Sum(child: Expression) extends 
AlgebraicAggregate {
 
   override val evaluateExpression = Cast(currentSum, resultType)
 }
+
+/**
+ * HyperLogLog++ is a state of the art cardinality estimation algorithm.
+ *
+ *
+ * This implementation has been based on the following papers:
+ * Papers:
+ * http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf
+ *
+ * HyperLogLog in Practice: Algorithmic Engineering of a State of
+ * The Art Cardinality Estimation Algorithm
+ *
+ * 
https://docs.google.com/document/d/1gyjfMHy43U9OWBXxfaeG-3MjGzejW1dlpyMwEYAAWEI/view?fullscreen#
+ *
+ * This implementation has been based on the following implementations:
+ * Note on provenance
+ * - Clearspring:
+ * - Aggregage Knowledge:
+ * - Algebird:
+ *
+ * Note on naming: Tried to match the paper.
+ *
+ *
+ * @param child
+ * @param relativeSD
+ */
+case class HyperLogLogPlusPlus(child: Expression, relativeSD: Double = 
0.05)
+extends AggregateFunction2 {
+  import HyperLogLogPlusPlus._
+
+  /**
+   * HLL++ uses 'b' bits for addressing. The more addressing bits we use, 
the more accurate the
+   * algorithm will be, and the more memory it will require. The 'b' value 
is based on the accuracy
+   * requested.
+   *
+   * HLL++ requires that we use at least 4 bits of addressing space (a 
minimum accuracy of 27%).
+   *
+   * TODO we currently round down to the nearest integer. This means 
accuracy is typically worse
+   * than the user expects.
+   */
+  private[this] val b = {
+(2.0d * Math.log(1.106d / relativeSD) / Math.log(2.0d)).toInt
+  }
+  require(b >= 4, "HLL++ requires at least 4 bits for addressing. " +
+"Use a higher accuracy, at least 27%.")
+
+  /**
+   * Shift used to extract the 'j' (the register) value from the hashed 
value.
+   *
+   * This assumes the use of 64-bit hashcodes.
+   */
+  private[this] val jShift = JLong.SIZE - b
+
+  /**
+   * Value to pad the 'w' value with before the number of leading zeros is 
determined.
+   */
+  private[this] val wPadding = 1L << (b - 1)
+
+  /**
+   * The number of registers used.
+   */
+  private[this] val m = 1 << b
+
+  /**
+   * The pre-calculated combination of: alpha * m * m
+   *
+   * 'alpha' corrects the raw cardinality estimate 'Z'. See the FlFuGaMe07 
paper for its
+   * derivation.
+   */
+  private[this] val alphaM2 = b match {
+case 4 => 0.673d * m * m
+case 5 => 0.697d * m * m
+case 6 => 0.709d * m * m
+case _ => (0.7213d / (1.0d + 1.079d / m)) * m * m
+  }
+
+  /**
+   * The number of words used to store the registers. We use Longs for 
storage because this is the
+   * most compact way of storage; Spark aligns to 8-byte words or uses 
Long wrappers.
+   *
+   * We only store whole registers per word in order to prevent overly 
complex bitwise operations.
+   * In practice this means we only use 60 out of 64 bits.
+   */
+  private[this] val numWords = m / REGISTERS_PER_WORD match {
+case x if m % REGISTERS_PER_WORD == 0 => x
+case x => x + 1
+  }
+
+  def children: Seq[Expression] = Seq(child)
+
+  def nullable: Boolean = false
+
+  def dataType: DataType = LongType
+
+  def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)
+
+  def bufferSchema: StructType = 
StructType.fromAttributes(bufferAttributes)
+
+  def cloneBufferAttributes: Seq[Attribute] = 
bufferAttributes.map(_.newInstance())
+
+  /** Allocate enough words to store all registers. */
+  val bufferAttributes: Seq[AttributeReference] = Seq.tabulate(numWords) { 
i =>
+AttributeReference(s"MS[$i]", LongType)()
+  }
+
+  /** Fill all words with zeros. */
+  def initialize(buffer: MutableRow): Unit = {
+var word = 0
+while (word < numWords) {
+  buffer.setLong(mutableBufferOffset + word, 0)
+  word += 1
+}
+  }
+
+  /**
+   * Update the HLL buffer.
+   *
+   * Variable names in the paper match variable names in the code.
+   */
+  def update(buffer: MutableRow, input: InternalRow): Unit = {
+val v = child.eval(input)
+if (v != null) {
+  // Create the hashed value 'x'.
+  val x = MurmurHash.hash64(v)
+
+  // Determine which register 'j' we are going to use.
+  val j = (x >>> jShift).toInt
+
+  // Determine the 

[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-29 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-144136673
  
We can work on improving the aggregate operator.


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

2015-09-28 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-143945761
  
@yhuai can we make non-codegen path use tungsten aggregate as well? 
Otherwise we would need to maintain two entirely separate codepath.


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

2015-09-28 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/8362#discussion_r40610006
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
 ---
@@ -302,3 +307,393 @@ case class Sum(child: Expression) extends 
AlgebraicAggregate {
 
   override val evaluateExpression = Cast(currentSum, resultType)
 }
+
+/**
+ * HyperLogLog++ is a state of the art cardinality estimation algorithm.
+ *
+ *
+ * This implementation has been based on the following papers:
+ * Papers:
+ * http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf
+ *
+ * HyperLogLog in Practice: Algorithmic Engineering of a State of
+ * The Art Cardinality Estimation Algorithm
+ *
+ * 
https://docs.google.com/document/d/1gyjfMHy43U9OWBXxfaeG-3MjGzejW1dlpyMwEYAAWEI/view?fullscreen#
+ *
+ * This implementation has been based on the following implementations:
+ * Note on provenance
+ * - Clearspring:
+ * - Aggregage Knowledge:
+ * - Algebird:
+ *
+ * Note on naming: Tried to match the paper.
+ *
+ *
+ * @param child
+ * @param relativeSD
+ */
+case class HyperLogLogPlusPlus(child: Expression, relativeSD: Double = 
0.05)
+extends AggregateFunction2 {
+  import HyperLogLogPlusPlus._
+
+  /**
+   * HLL++ uses 'b' bits for addressing. The more addressing bits we use, 
the more accurate the
+   * algorithm will be, and the more memory it will require. The 'b' value 
is based on the accuracy
+   * requested.
+   *
+   * HLL++ requires that we use at least 4 bits of addressing space (a 
minimum accuracy of 27%).
+   *
+   * TODO we currently round down to the nearest integer. This means 
accuracy is typically worse
--- End diff --

Should we round it up?


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

2015-09-28 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/8362#discussion_r40610794
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
 ---
@@ -302,3 +307,393 @@ case class Sum(child: Expression) extends 
AlgebraicAggregate {
 
   override val evaluateExpression = Cast(currentSum, resultType)
 }
+
+/**
+ * HyperLogLog++ is a state of the art cardinality estimation algorithm.
+ *
+ *
+ * This implementation has been based on the following papers:
+ * Papers:
+ * http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf
+ *
+ * HyperLogLog in Practice: Algorithmic Engineering of a State of
+ * The Art Cardinality Estimation Algorithm
+ *
+ * 
https://docs.google.com/document/d/1gyjfMHy43U9OWBXxfaeG-3MjGzejW1dlpyMwEYAAWEI/view?fullscreen#
+ *
+ * This implementation has been based on the following implementations:
+ * Note on provenance
+ * - Clearspring:
+ * - Aggregage Knowledge:
+ * - Algebird:
+ *
+ * Note on naming: Tried to match the paper.
+ *
+ *
+ * @param child
+ * @param relativeSD
+ */
+case class HyperLogLogPlusPlus(child: Expression, relativeSD: Double = 
0.05)
+extends AggregateFunction2 {
+  import HyperLogLogPlusPlus._
+
+  /**
+   * HLL++ uses 'b' bits for addressing. The more addressing bits we use, 
the more accurate the
+   * algorithm will be, and the more memory it will require. The 'b' value 
is based on the accuracy
+   * requested.
+   *
+   * HLL++ requires that we use at least 4 bits of addressing space (a 
minimum accuracy of 27%).
+   *
+   * TODO we currently round down to the nearest integer. This means 
accuracy is typically worse
+   * than the user expects.
+   */
+  private[this] val b = {
+(2.0d * Math.log(1.106d / relativeSD) / Math.log(2.0d)).toInt
+  }
+  require(b >= 4, "HLL++ requires at least 4 bits for addressing. " +
+"Use a higher accuracy, at least 27%.")
+
+  /**
+   * Shift used to extract the 'j' (the register) value from the hashed 
value.
+   *
+   * This assumes the use of 64-bit hashcodes.
+   */
+  private[this] val jShift = JLong.SIZE - b
+
+  /**
+   * Value to pad the 'w' value with before the number of leading zeros is 
determined.
+   */
+  private[this] val wPadding = 1L << (b - 1)
+
+  /**
+   * The number of registers used.
+   */
+  private[this] val m = 1 << b
+
+  /**
+   * The pre-calculated combination of: alpha * m * m
+   *
+   * 'alpha' corrects the raw cardinality estimate 'Z'. See the FlFuGaMe07 
paper for its
+   * derivation.
+   */
+  private[this] val alphaM2 = b match {
+case 4 => 0.673d * m * m
+case 5 => 0.697d * m * m
+case 6 => 0.709d * m * m
+case _ => (0.7213d / (1.0d + 1.079d / m)) * m * m
+  }
+
+  /**
+   * The number of words used to store the registers. We use Longs for 
storage because this is the
+   * most compact way of storage; Spark aligns to 8-byte words or uses 
Long wrappers.
+   *
+   * We only store whole registers per word in order to prevent overly 
complex bitwise operations.
+   * In practice this means we only use 60 out of 64 bits.
+   */
+  private[this] val numWords = m / REGISTERS_PER_WORD match {
+case x if m % REGISTERS_PER_WORD == 0 => x
--- End diff --

This will not happen here, since m is 1<

[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-28 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-143902631
  
I took a round, this looks pretty good to me over all. 

Currently, each grouping key needs 200 bytes (b=8, by default), so the 
sparse version could help to reduce the memory usage in case of the average 
number of distinct value is small (I believe it's a common case). Since we 
already support external aggregation (using sort based), so it's not critical, 
could be a optional improvement (separated PR).

Had ran a small benchmark for this patch, I'm surprised it's slower than 
1.5 (using old aggregation). The test code:
```
df = sqlContext.range(1<<25).agg(approxCountDistinct("id"))
df.explain()
t = time.time()
print df.collect()
print time.time() - t
```
It took 3.4 seconds in 1.5, but 6.4 seconds with this patch.

The plain in 1.5:
```
Aggregate false, [APPROXIMATE COUNT(DISTINCT PartialApproxCountDistinct#2) 
AS APPROXIMATE COUNT(DISTINCT id)#1L]
 Exchange SinglePartition
  Aggregate true, [APPROXIMATE COUNT(DISTINCT id#0L) AS 
PartialApproxCountDistinct#2]
   Scan PhysicalRDD[id#0L]
```

The plan with this patch:
```
SortBasedAggregate(key=[], 
functions=[(hyperloglogplusplus(id#0L),mode=Final,isDistinct=false)], 
output=[APPROXIMATE COUNT(DISTINCT id)#1L])
 ConvertToSafe
  TungstenExchange SinglePartition
   ConvertToUnsafe
SortBasedAggregate(key=[], 
functions=[(hyperloglogplusplus(id#0L),mode=Partial,isDistinct=false)], 
output=[MS[0]#30L,MS[1]#31L,MS[2]#32L,MS[3]#33L,MS[4]#34L,MS[5]#35L,MS[6]#36L,MS[7]#37L,MS[8]#38L,MS[9]#39L,MS[10]#40L,MS[11]#41L,MS[12]#42L,MS[13]#43L,MS[14]#44L,MS[15]#45L,MS[16]#46L,MS[17]#47L,MS[18]#48L,MS[19]#49L,MS[20]#50L,MS[21]#51L,MS[22]#52L,MS[23]#53L,MS[24]#54L,MS[25]#55L])
 Scan PhysicalRDD[id#0L]
```

Discussed this with @yhuai , the slowness may come from the new 
aggregation, that only not support AAlgebraicAggregate in hash mode, we will 
fix that in 1.6.


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

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



[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-22 Thread hvanhovell
Github user hvanhovell commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-142323666
  
@MLnick I am in the process of moving house, so I am a bit slow/late with 
my response :(... 

I think it is very usefull to be able to return the HLL registers to the 
users (it could also be nice to use in cost based planning). I would rather 
give it a different name though ```createHLLRegisters``` for instance (the name 
needs work), to make it clear that we are doing something different.

The UDAF should support a rsd parameter. Doesn't it? I'll add a test.


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

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



[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-14 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-140141999
  
@MLnick This one will replace the existing implementation. For now, we will 
do conversion as shown at 
https://github.com/apache/spark/pull/8362/files#diff-78b9b210b8cee72e7097bc1af44bd315L98.
 Later, we will remove the old implementation (`AggregateFunction1` interface).


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

2015-09-14 Thread MLnick
Github user MLnick commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-140019207
  
@hvanhovell as discussed on the `dev` mailing list, perhaps it would be 
interesting to allow the return type to include the aggregated HLL registers. 
This could be (for example) in the form of `StructType` `{'cardinality':Long, 
'hll': Array[Byte]`}, where the `hll` is in the same serialized form that can 
be used to instantiate say a `StreamLib` or `Algebird` HLL class for use 
outside of Spark.

Is it possible to specify input arguments for `rsd`? So `SELECT APPROX 
DISTINCT(column, 0.1) FROM ...`? If so, then another option is to add a further 
argument such as `returnHLL: Boolean = false` so that either the raw HLL or the 
cardinality is returned? 


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

2015-09-14 Thread MLnick
Github user MLnick commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-140019691
  
@hvanhovell @rxin is it intended that this replace the existing 
`approxCountDistinct ` implementation? And I assume this will happen 
automatically due to extending `AggregateFunction2 `?


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

2015-09-10 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-139406671
  
One quick note: https://github.com/twitter/algebird/pull/491/files

anything we can learn from the above 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 pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-09-08 Thread hvanhovell
Github user hvanhovell commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-138493630
  
@rxin the dense version of HLL++ is ready. We could also add this, and add 
the sparse logic in a follow-up PR. Let me know what you think. I'll close if 
you'd rather do everything in one go.


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

2015-09-08 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-138699022
  
Ah ok. will add this to our sprint backlog and get somebody to review it 
soon.



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

2015-09-07 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-138384295
  
@hvanhovell do you mind closing this pull request, and re-open when you 
feel it is ready for review again?


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

2015-08-27 Thread hvanhovell
Github user hvanhovell commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-135619245
  
Implemented initial non-sparse HLL++. I am going to take a look at the 
sparse version next week. The results are still equal to the Clearspring HLL+ 
implementation in non-sparse mode.

I also need to clean-up the docs for the main HLL++ class a bit.


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

2015-08-27 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-135639256
  
  [Test build #41719 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41719/console)
 for   PR 8362 at commit 
[`1ea722b`](https://github.com/apache/spark/commit/1ea722b44745036ef568447f9db93a7ebade8b12).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class HyperLogLogPlusPlus(child: Expression, relativeSD: Double = 
0.05)`



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

2015-08-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-135617993
  
Merged build started.


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

2015-08-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-135617983
  
 Merged build triggered.


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

2015-08-27 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-135619890
  
  [Test build #41719 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41719/consoleFull)
 for   PR 8362 at commit 
[`1ea722b`](https://github.com/apache/spark/commit/1ea722b44745036ef568447f9db93a7ebade8b12).


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

2015-08-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-135639349
  
Merged build finished. Test PASSed.


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

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



[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-08-27 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-135639351
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41719/
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: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-08-21 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133544919
  
This made my day. The approach is super cool.

Couple suggestions:

1. Can we use HyperLogLogPlus? It's also in streamlib: 
https://github.com/addthis/stream-lib/blob/master/src/main/java/com/clearspring/analytics/stream/cardinality/HyperLogLogPlus.java

2. Can we write this in a way to make it more unit testable?


Beyond this, would be cool to have count-min sketch too! (future 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: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-08-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133562454
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41377/
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: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-08-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133562302
  
  [Test build #41377 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41377/console)
 for   PR 8362 at commit 
[`e178d9e`](https://github.com/apache/spark/commit/e178d9e942720c879dc947a5fd3e6593a351f04f).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class HyperLogLog(child: Expression, relativeSD: Double = 0.05)`



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

2015-08-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133562453
  
Merged build finished. Test PASSed.


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

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



[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-08-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133515988
  
Can one of the admins verify this patch?


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

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



[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-08-21 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133523761
  
ok to test


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

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



[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-08-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133523850
  
Merged build started.


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

2015-08-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133523827
  
 Merged build triggered.


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

2015-08-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133525793
  
  [Test build #41377 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41377/consoleFull)
 for   PR 8362 at commit 
[`e178d9e`](https://github.com/apache/spark/commit/e178d9e942720c879dc947a5fd3e6593a351f04f).


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

2015-08-21 Thread hvanhovell
GitHub user hvanhovell opened a pull request:

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

[SPARK-9741][SQL] Approximate Count Distinct using the new UDAF interface.

This PR implements a HyperLogLog based Approximate Count Distinct function 
using the new UDAF interface.

The implementation is inspired by the ClearSpring HyperLogLog 
implementation and should produce the same results.

There is still some documentation and testing left to do.

cc @yhuai

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

$ git pull https://github.com/hvanhovell/spark SPARK-9741

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

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


commit f52de0236fbfb85053bd6538b12025ba23d34ee1
Author: Herman van Hovell hvanhov...@questtec.nl
Date:   2015-08-21T02:32:15Z

Created HyperLogLog aggregate.

commit 8ec27b93da4f2d39da7c72f7a88c2742cd38318a
Author: Herman van Hovell hvanhov...@questtec.nl
Date:   2015-08-21T12:57:27Z

Added HLL to conversions. More doc. Improvement.

commit e178d9e942720c879dc947a5fd3e6593a351f04f
Author: Herman van Hovell hvanhov...@questtec.nl
Date:   2015-08-21T17:56:39Z

Bug fixes. Style. Documentation.




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

2015-08-21 Thread hvanhovell
Github user hvanhovell commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133566608
  
Thanks.

I was aiming for compatibility with the existing approxCountDistinct, but 
we can also implement HLL++. HLL++ introduces three (orthogonal) refinements: 
64-bit hashing, better low cardinality corrections and a sparse encoding 
scheme. The first two refinements are easy to add. The third will require a bit 
more effort.

Unit testing this is a bit of a challenge. End-to-end (blackbox) testing is 
no problem, as long as we know what the result should be, or if we do random 
testing (results should be within 5% of the actual value). Testing parts of the 
algorithm is a bit of a PITA:
* It is hard to reason about the results (the updated registers) HLL 
produces.
* Register access code and HLL code are intertwined.

Both the 
[ClearSpring](https://github.com/addthis/stream-lib/blob/master/src/main/java/com/clearspring/analytics/stream/cardinality/HyperLogLog.java)
 and 
[AggregateKnowledge](https://github.com/aggregateknowledge/java-hll/blob/master/src/main/java/net/agkn/hll/HLL.java)
 implementations resort to blackbox testing. I will create some blackbox tests.


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

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



[GitHub] spark pull request: [SPARK-9741][SQL] Approximate Count Distinct u...

2015-08-21 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/8362#issuecomment-133567281
  
Thanks - I think blackbox testing is fine. But it would be great to apply 
that at the unit testing level, i.e. running directly against the aggregate 
function, rather than against Spark SQL end to end.



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

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