Skip to content

Commit

Permalink
Merge branch 'master' of git://git.apache.org/spark into metrics-stru…
Browse files Browse the repository at this point in the history
…cture-improvement
  • Loading branch information
sarutak committed Sep 4, 2014
2 parents 4603a39 + 1bed0a3 commit 3e098d8
Show file tree
Hide file tree
Showing 29 changed files with 534 additions and 464 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ class LeastSquaresGradient extends Gradient {
class HingeGradient extends Gradient {
override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = {
val dotProduct = dot(data, weights)
// Our loss function with {0, 1} labels is max(0, 1 - (2y 1) (f_w(x)))
// Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x)))
// Therefore the gradient is -(2y - 1)*x
val labelScaled = 2 * label - 1.0
if (1.0 > labelScaled * dotProduct) {
Expand All @@ -146,7 +146,7 @@ class HingeGradient extends Gradient {
weights: Vector,
cumGradient: Vector): Double = {
val dotProduct = dot(data, weights)
// Our loss function with {0, 1} labels is max(0, 1 - (2y 1) (f_w(x)))
// Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x)))
// Therefore the gradient is -(2y - 1)*x
val labelScaled = 2 * label - 1.0
if (1.0 > labelScaled * dotProduct) {
Expand Down
2 changes: 2 additions & 0 deletions python/pyspark/shell.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
sys.exit(1)


import atexit
import os
import platform
import pyspark
Expand All @@ -42,6 +43,7 @@
SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"])

sc = SparkContext(appName="PySparkShell", pyFiles=add_files)
atexit.register(lambda: sc.stop())

print("""Welcome to
____ __
Expand Down
17 changes: 9 additions & 8 deletions python/pyspark/sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -942,9 +942,7 @@ def __init__(self, sparkContext, sqlContext=None):
self._jsc = self._sc._jsc
self._jvm = self._sc._jvm
self._pythonToJava = self._jvm.PythonRDD.pythonToJavaArray

if sqlContext:
self._scala_SQLContext = sqlContext
self._scala_SQLContext = sqlContext

@property
def _ssql_ctx(self):
Expand All @@ -953,7 +951,7 @@ def _ssql_ctx(self):
Subclasses can override this property to provide their own
JVM Contexts.
"""
if not hasattr(self, '_scala_SQLContext'):
if self._scala_SQLContext is None:
self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc())
return self._scala_SQLContext

Expand All @@ -970,23 +968,26 @@ def registerFunction(self, name, f, returnType=StringType()):
>>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType())
>>> sqlCtx.sql("SELECT stringLengthInt('test')").collect()
[Row(c0=4)]
>>> sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType())
>>> sqlCtx.sql("SELECT twoArgs('test', 1)").collect()
[Row(c0=5)]
"""
func = lambda _, it: imap(lambda x: f(*x), it)
command = (func,
BatchedSerializer(PickleSerializer(), 1024),
BatchedSerializer(PickleSerializer(), 1024))
pickled_command = CloudPickleSerializer().dumps(command)
broadcast_vars = ListConverter().convert(
[x._jbroadcast for x in self._sc._pickled_broadcast_vars],
self._sc._gateway._gateway_client)
self._sc._pickled_broadcast_vars.clear()
env = MapConverter().convert(self._sc.environment,
self._sc._gateway._gateway_client)
includes = ListConverter().convert(self._sc._python_includes,
self._sc._gateway._gateway_client)
self._ssql_ctx.registerPython(name,
bytearray(CloudPickleSerializer().dumps(command)),
bytearray(pickled_command),
env,
includes,
self._sc.pythonExec,
broadcast_vars,
self._sc._javaAccumulator,
str(returnType))

Expand Down
22 changes: 22 additions & 0 deletions python/pyspark/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
from pyspark.files import SparkFiles
from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer
from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter
from pyspark.sql import SQLContext, IntegerType

_have_scipy = False
_have_numpy = False
Expand Down Expand Up @@ -525,6 +526,27 @@ def test_histogram(self):
self.assertRaises(TypeError, lambda: rdd.histogram(2))


class TestSQL(PySparkTestCase):

def setUp(self):
PySparkTestCase.setUp(self)
self.sqlCtx = SQLContext(self.sc)

def test_udf(self):
self.sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType())
[row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect()
self.assertEqual(row[0], 5)

def test_broadcast_in_udf(self):
bar = {"a": "aa", "b": "bb", "c": "abc"}
foo = self.sc.broadcast(bar)
self.sqlCtx.registerFunction("MYUDF", lambda x: foo.value[x] if x else '')
[res] = self.sqlCtx.sql("SELECT MYUDF('c')").collect()
self.assertEqual("abc", res[0])
[res] = self.sqlCtx.sql("SELECT MYUDF('')").collect()
self.assertEqual("", res[0])


class TestIO(PySparkTestCase):

def test_stdout_redirection(self):
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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

/**
* Builds a map that is keyed by an Attribute's expression id. Using the expression id allows values
* to be looked up even when the attributes used differ cosmetically (i.e., the capitalization
* of the name, or the expected nullability).
*/
object AttributeMap {
def apply[A](kvs: Seq[(Attribute, A)]) =
new AttributeMap(kvs.map(kv => (kv._1.exprId, (kv._1, kv._2))).toMap)
}

class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)])
extends Map[Attribute, A] with Serializable {

override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2)

override def + [B1 >: A](kv: (Attribute, B1)): Map[Attribute, B1] =
(baseMap.map(_._2) + kv).toMap

override def iterator: Iterator[(Attribute, A)] = baseMap.map(_._2).iterator

override def -(key: Attribute): Map[Attribute, A] = (baseMap.map(_._2) - key).toMap
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,12 +38,20 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean)
}

object BindReferences extends Logging {
def bindReference[A <: Expression](expression: A, input: Seq[Attribute]): A = {

def bindReference[A <: Expression](
expression: A,
input: Seq[Attribute],
allowFailures: Boolean = false): A = {
expression.transform { case a: AttributeReference =>
attachTree(a, "Binding attribute") {
val ordinal = input.indexWhere(_.exprId == a.exprId)
if (ordinal == -1) {
sys.error(s"Couldn't find $a in ${input.mkString("[", ",", "]")}")
if (allowFailures) {
a
} else {
sys.error(s"Couldn't find $a in ${input.mkString("[", ",", "]")}")
}
} else {
BoundReference(ordinal, a.dataType, a.nullable)
}
Expand Down
7 changes: 7 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import java.util.Properties
private[spark] object SQLConf {
val COMPRESS_CACHED = "spark.sql.inMemoryColumnarStorage.compressed"
val COLUMN_BATCH_SIZE = "spark.sql.inMemoryColumnarStorage.batchSize"
val IN_MEMORY_PARTITION_PRUNING = "spark.sql.inMemoryColumnarStorage.partitionPruning"
val AUTO_BROADCASTJOIN_THRESHOLD = "spark.sql.autoBroadcastJoinThreshold"
val DEFAULT_SIZE_IN_BYTES = "spark.sql.defaultSizeInBytes"
val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions"
Expand Down Expand Up @@ -124,6 +125,12 @@ trait SQLConf {
private[spark] def isParquetBinaryAsString: Boolean =
getConf(PARQUET_BINARY_AS_STRING, "false").toBoolean

/**
* When set to true, partition pruning for in-memory columnar tables is enabled.
*/
private[spark] def inMemoryPartitionPruning: Boolean =
getConf(IN_MEMORY_PARTITION_PRUNING, "false").toBoolean

/** ********************** SQLConf functionality methods ************ */

/** Set Spark SQL configuration properties. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark.sql
import java.util.{List => JList, Map => JMap}

import org.apache.spark.Accumulator
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.sql.catalyst.ScalaReflection
import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf}
import org.apache.spark.sql.execution.PythonUDF
Expand All @@ -38,6 +39,7 @@ protected[sql] trait UDFRegistration {
envVars: JMap[String, String],
pythonIncludes: JList[String],
pythonExec: String,
broadcastVars: JList[Broadcast[Array[Byte]]],
accumulator: Accumulator[JList[Array[Byte]]],
stringDataType: String): Unit = {
log.debug(
Expand All @@ -61,6 +63,7 @@ protected[sql] trait UDFRegistration {
envVars,
pythonIncludes,
pythonExec,
broadcastVars,
accumulator,
dataType,
e)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ private[sql] trait ColumnBuilder {
/**
* Column statistics information
*/
def columnStats: ColumnStats[_, _]
def columnStats: ColumnStats

/**
* Returns the final columnar byte buffer.
Expand All @@ -47,7 +47,7 @@ private[sql] trait ColumnBuilder {
}

private[sql] class BasicColumnBuilder[T <: DataType, JvmType](
val columnStats: ColumnStats[T, JvmType],
val columnStats: ColumnStats,
val columnType: ColumnType[T, JvmType])
extends ColumnBuilder {

Expand Down Expand Up @@ -81,18 +81,18 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType](

private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType](
columnType: ColumnType[T, JvmType])
extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType)
extends BasicColumnBuilder[T, JvmType](new NoopColumnStats, columnType)
with NullableColumnBuilder

private[sql] abstract class NativeColumnBuilder[T <: NativeType](
override val columnStats: NativeColumnStats[T],
override val columnStats: ColumnStats,
override val columnType: NativeColumnType[T])
extends BasicColumnBuilder[T, T#JvmType](columnStats, columnType)
with NullableColumnBuilder
with AllCompressionSchemes
with CompressibleColumnBuilder[T]

private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN)
private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new NoopColumnStats, BOOLEAN)

private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT)

Expand Down
Loading

0 comments on commit 3e098d8

Please sign in to comment.