Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
28 changes: 28 additions & 0 deletions python/pyspark/sql/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,12 @@ def _():
'measured in radians.',
}

_functions_2_2 = {
'to_date': 'Converts a string date into a DateType using the (optionally) specified format.',
'to_timestamp': 'Converts a string timestamp into a timestamp type using the ' +
'(optionally) specified format.',
}

# math functions that take two arguments as input
_binary_mathfunctions = {
'atan2': 'Returns the angle theta from the conversion of rectangular coordinates (x, y) to' +
Expand Down Expand Up @@ -350,6 +356,28 @@ def countDistinct(col, *cols):
return Column(jc)


def every(col):
"""Aggregate function: returns true if all values in a group are true.
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.every(_to_java_column(col))
return Column(jc)


def any(col):
"""Aggregate function: returns true if at least one value in the group is true.
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.any(_to_java_column(col))
return Column(jc)


def some(col):
"""Aggregate function: returns true if at least one value in the group is true.
"""
return any(col)


@since(1.3)
def first(col, ignorenulls=False):
"""Aggregate function: returns the first value in a group.
Expand Down
87 changes: 87 additions & 0 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -1158,6 +1158,24 @@ def test_cov(self):
cov = df.stat.cov("a", "b")
self.assertTrue(abs(cov - 55.0 / 3) < 1e-6)

def test_every_any(self):
from pyspark.sql import functions
data = [
Row(key="a", value=False),
Row(key="a", value=True),
Row(key="a", value=False),
Row(key="b", value=True),
Row(key="b", value=True),
Row(key="c", value=False),
Row(key="d", value=True),
Row(key="d", value=None)
]
df = self.sc.parallelize(data).toDF()
df2 = df.select(functions.every(df.value).alias('a'),
functions.any(df.value).alias('b'),
functions.some(df.value).alias('c'))
self.assertEqual([Row(a=False, b=True, c=True)], df2.collect())

def test_crosstab(self):
df = self.sc.parallelize([Row(a=i % 3, b=i % 2) for i in range(1, 7)]).toDF()
ct = df.stat.crosstab("a", "b").collect()
Expand Down Expand Up @@ -2631,6 +2649,75 @@ def test_window_functions_cumulative_sum(self):
for r, ex in zip(rs, expected):
self.assertEqual(tuple(r), ex[:len(r)])

def test_window_functions_every_any(self):
df = self.spark.createDataFrame([
("a", False),
("a", True),
("a", False),
("b", True),
("b", True),
("c", False),
("d", True),
("d", None)
], ["key", "value"])
w = Window \
.partitionBy("key").orderBy("value") \
.rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)
from pyspark.sql import functions as F
sel = df.select(df.key,
df.value,
F.every("value").over(w),
F.any("value").over(w),
F.some("value").over(w))
rs = sel.collect()
expected = [
("a", False, False, True, True),
("a", False, False, True, True),
("a", True, False, True, True),
("b", True, True, True, True),
("b", True, True, True, True),
("c", False, False, False, False),
("d", None, False, True, True),
("d", True, False, True, True)
]
for r, ex in zip(rs, expected):
self.assertEqual(tuple(r), ex[:len(r)])

def test_window_functions_every_any_without_partitionBy(self):
df = self.spark.createDataFrame([
(False,),
(True,),
(False,),
(True,),
(True,),
(False,),
(True,),
(None,)
], ["value"])
w1 = Window.orderBy("value").rowsBetween(Window.unboundedPreceding, 0)
w2 = Window.orderBy("value").rowsBetween(-1, 0)
from pyspark.sql import functions as F
sel = df.select(df.value,
F.every("value").over(w1),
F.any("value").over(w1),
F.some("value").over(w1),
F.every("value").over(w2),
F.any("value").over(w2),
F.some("value").over(w2))
rs = sel.collect()
expected = [
(None, False, False, False, False, False, False),
(False, False, False, False, False, False, False),
(False, False, False, False, False, False, False),
(False, False, False, False, False, False, False),
(True, False, True, True, False, True, True),
(True, False, True, True, True, True, True),
(True, False, True, True, True, True, True),
(True, False, True, True, True, True, True)
]
for r, ex in zip(rs, expected):
self.assertEqual(tuple(r), ex[:len(r)])

def test_collect_functions(self):
df = self.spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"])
from pyspark.sql import functions
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,9 @@ object FunctionRegistry {
expression[CollectList]("collect_list"),
expression[CollectSet]("collect_set"),
expression[CountMinSketchAgg]("count_min_sketch"),
expression[Every]("every"),
expression[AnyAgg]("any"),
expression[AnyAgg]("some"),

// string functions
expression[Ascii]("ascii"),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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 org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.util.TypeUtils
import org.apache.spark.sql.types._

@ExpressionDescription(
usage = "_FUNC_(expr) - Returns true if at least one value of `expr` is true.")
case class AnyAgg(child: Expression) extends DeclarativeAggregate with ImplicitCastInputTypes {

override def children: Seq[Expression] = child :: Nil

override def nullable: Boolean = true

override def dataType: DataType = BooleanType

override def inputTypes: Seq[AbstractDataType] = Seq(BooleanType)

override def checkInputDataTypes(): TypeCheckResult =
TypeUtils.checkForBooleanExpr(child.dataType, "function any")

private lazy val some = AttributeReference("some", BooleanType)()

private lazy val emptySet = AttributeReference("emptySet", BooleanType)()

override lazy val aggBufferAttributes = some :: emptySet :: Nil

override lazy val initialValues: Seq[Expression] = Seq(
Literal(false),
Literal(true)
)

override lazy val updateExpressions: Seq[Expression] = Seq(
Or(some, Coalesce(Seq(child, Literal(false)))),
Literal(false)
)

override lazy val mergeExpressions: Seq[Expression] = Seq(
Or(some.left, some.right),
And(emptySet.left, emptySet.right)
)

override lazy val evaluateExpression: Expression = And(!emptySet, some)
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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 org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.util.TypeUtils
import org.apache.spark.sql.types._

@ExpressionDescription(
usage = "_FUNC_(expr) - Returns true if all values of `expr` are true.")
case class Every(child: Expression) extends DeclarativeAggregate with ImplicitCastInputTypes {

override def children: Seq[Expression] = child :: Nil

override def nullable: Boolean = true

override def dataType: DataType = BooleanType

override def inputTypes: Seq[AbstractDataType] = Seq(BooleanType)

override def checkInputDataTypes(): TypeCheckResult =
TypeUtils.checkForBooleanExpr(child.dataType, "function every")

private lazy val every = AttributeReference("every", BooleanType)()

private lazy val emptySet = AttributeReference("emptySet", BooleanType)()

override lazy val aggBufferAttributes = every :: emptySet :: Nil

override lazy val initialValues: Seq[Expression] = Seq(
Literal(true),
Literal(true)
)

override lazy val updateExpressions: Seq[Expression] = Seq(
And(every, Coalesce(Seq(child, Literal(false)))),
Literal(false)
)

override lazy val mergeExpressions: Seq[Expression] = Seq(
And(every.left, every.right),
And(emptySet.left, emptySet.right)
)

override lazy val evaluateExpression: Expression = And(!emptySet, every)
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,14 @@ object TypeUtils {
}
}

def checkForBooleanExpr(dt: DataType, caller: String): TypeCheckResult = {
if (dt.isInstanceOf[BooleanType] || dt == NullType) {
TypeCheckResult.TypeCheckSuccess
} else {
TypeCheckResult.TypeCheckFailure(s"$caller requires boolean types, not $dt")
}
}

def checkForOrderingExpr(dt: DataType, caller: String): TypeCheckResult = {
if (RowOrdering.isOrderable(dt)) {
TypeCheckResult.TypeCheckSuccess
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,8 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite {
assertSuccess(Sum('stringField))
assertSuccess(Average('stringField))
assertSuccess(Min('arrayField))
assertSuccess(Every('booleanField))
assertSuccess(AnyAgg('booleanField))

assertError(Min('mapField), "min does not support ordering on type")
assertError(Max('mapField), "max does not support ordering on type")
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* 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 org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Literal}
import org.apache.spark.sql.types.BooleanType

class AnyTestSuite extends SparkFunSuite {
val input = AttributeReference("input", BooleanType, nullable = true)()
val evaluator = DeclarativeAggregateEvaluator(AnyAgg(input), Seq(input))

test("empty buffer") {
assert(evaluator.initialize() === InternalRow(false, true))
}

test("update") {
val result = evaluator.update(
InternalRow(true),
InternalRow(false),
InternalRow(true))
assert(result === InternalRow(true, false))
}

test("merge") {
// Empty merge
val p0 = evaluator.initialize()
assert(evaluator.merge(p0) === InternalRow(false, true))

// Single merge
val p1 = evaluator.update(InternalRow(true), InternalRow(true))
assert(evaluator.merge(p1) === InternalRow(true, false))

// Multiple merges.
val p2 = evaluator.update(InternalRow(false), InternalRow(null))
assert(evaluator.merge(p1, p2) === InternalRow(true, false))

// Empty partitions (p0 is empty)
assert(evaluator.merge(p0, p2) === InternalRow(false, false))
assert(evaluator.merge(p2, p1, p0) === InternalRow(true, false))
}

test("eval") {
// Null Eval
assert(evaluator.eval(InternalRow(null, true)) === InternalRow(false))

// Empty Eval
val p0 = evaluator.initialize()
assert(evaluator.eval(p0) === InternalRow(false))

// Update - Eval
val p1 = evaluator.update(InternalRow(true), InternalRow(null))
assert(evaluator.eval(p1) === InternalRow(true))

// Update - Merge - Eval
val p2 = evaluator.update(InternalRow(false), InternalRow(false))
val m1 = evaluator.merge(p0, p2)
assert(evaluator.eval(m1) === InternalRow(false))

// Update - Merge - Eval (empty partition at the end)
val m2 = evaluator.merge(p2, p1, p0)
assert(evaluator.eval(m2) === InternalRow(true))
}

}
Loading