Skip to content

Commit 9969c14

Browse files
committed
Merge remote-tracking branch 'origin/master' into sql-external-sort
2 parents 5822e6f + 132e7fc commit 9969c14

File tree

17 files changed

+672
-175
lines changed

17 files changed

+672
-175
lines changed

R/README.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,7 @@ SparkR is an R package that provides a light-weight frontend to use Spark from R
66

77
#### Build Spark
88

9-
Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-PsparkR` profile to build the R package. For example to use the default Hadoop versions you can run
9+
Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run
1010
```
1111
build/mvn -DskipTests -Psparkr package
1212
```
Lines changed: 107 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,107 @@
1+
#
2+
# Licensed to the Apache Software Foundation (ASF) under one or more
3+
# contributor license agreements. See the NOTICE file distributed with
4+
# this work for additional information regarding copyright ownership.
5+
# The ASF licenses this file to You under the Apache License, Version 2.0
6+
# (the "License"); you may not use this file except in compliance with
7+
# the License. You may obtain a copy of the License at
8+
#
9+
# http://www.apache.org/licenses/LICENSE-2.0
10+
#
11+
# Unless required by applicable law or agreed to in writing, software
12+
# distributed under the License is distributed on an "AS IS" BASIS,
13+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
# See the License for the specific language governing permissions and
15+
# limitations under the License.
16+
#
17+
18+
# For this example, we shall use the "flights" dataset
19+
# The dataset consists of every flight departing Houston in 2011.
20+
# The data set is made up of 227,496 rows x 14 columns.
21+
22+
# To run this example use
23+
# ./bin/sparkR --packages com.databricks:spark-csv_2.10:1.0.3
24+
# examples/src/main/r/data-manipulation.R <path_to_csv>
25+
26+
# Load SparkR library into your R session
27+
library(SparkR)
28+
29+
args <- commandArgs(trailing = TRUE)
30+
31+
if (length(args) != 1) {
32+
print("Usage: data-manipulation.R <path-to-flights.csv")
33+
print("The data can be downloaded from: http://s3-us-west-2.amazonaws.com/sparkr-data/flights.csv ")
34+
q("no")
35+
}
36+
37+
## Initialize SparkContext
38+
sc <- sparkR.init(appName = "SparkR-data-manipulation-example")
39+
40+
## Initialize SQLContext
41+
sqlContext <- sparkRSQL.init(sc)
42+
43+
flightsCsvPath <- args[[1]]
44+
45+
# Create a local R dataframe
46+
flights_df <- read.csv(flightsCsvPath, header = TRUE)
47+
flights_df$date <- as.Date(flights_df$date)
48+
49+
## Filter flights whose destination is San Francisco and write to a local data frame
50+
SFO_df <- flights_df[flights_df$dest == "SFO", ]
51+
52+
# Convert the local data frame into a SparkR DataFrame
53+
SFO_DF <- createDataFrame(sqlContext, SFO_df)
54+
55+
# Directly create a SparkR DataFrame from the source data
56+
flightsDF <- read.df(sqlContext, flightsCsvPath, source = "com.databricks.spark.csv", header = "true")
57+
58+
# Print the schema of this Spark DataFrame
59+
printSchema(flightsDF)
60+
61+
# Cache the DataFrame
62+
cache(flightsDF)
63+
64+
# Print the first 6 rows of the DataFrame
65+
showDF(flightsDF, numRows = 6) ## Or
66+
head(flightsDF)
67+
68+
# Show the column names in the DataFrame
69+
columns(flightsDF)
70+
71+
# Show the number of rows in the DataFrame
72+
count(flightsDF)
73+
74+
# Select specific columns
75+
destDF <- select(flightsDF, "dest", "cancelled")
76+
77+
# Using SQL to select columns of data
78+
# First, register the flights DataFrame as a table
79+
registerTempTable(flightsDF, "flightsTable")
80+
destDF <- sql(sqlContext, "SELECT dest, cancelled FROM flightsTable")
81+
82+
# Use collect to create a local R data frame
83+
local_df <- collect(destDF)
84+
85+
# Print the newly created local data frame
86+
head(local_df)
87+
88+
# Filter flights whose destination is JFK
89+
jfkDF <- filter(flightsDF, "dest = \"JFK\"") ##OR
90+
jfkDF <- filter(flightsDF, flightsDF$dest == "JFK")
91+
92+
# If the magrittr library is available, we can use it to
93+
# chain data frame operations
94+
if("magrittr" %in% rownames(installed.packages())) {
95+
library(magrittr)
96+
97+
# Group the flights by date and then find the average daily delay
98+
# Write the result into a DataFrame
99+
groupBy(flightsDF, flightsDF$date) %>%
100+
summarize(avg(flightsDF$dep_delay), avg(flightsDF$arr_delay)) -> dailyDelayDF
101+
102+
# Print the computed data frame
103+
head(dailyDelayDF)
104+
}
105+
106+
# Stop the SparkContext now
107+
sparkR.stop()

python/pyspark/sql/functions.py

Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -395,6 +395,34 @@ def randn(seed=None):
395395
return Column(jc)
396396

397397

398+
@ignore_unicode_prefix
399+
@since(1.5)
400+
def hex(col):
401+
"""Computes hex value of the given column, which could be StringType,
402+
BinaryType, IntegerType or LongType.
403+
404+
>>> sqlContext.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect()
405+
[Row(hex(a)=u'414243', hex(b)=u'3')]
406+
"""
407+
sc = SparkContext._active_spark_context
408+
jc = sc._jvm.functions.hex(_to_java_column(col))
409+
return Column(jc)
410+
411+
412+
@ignore_unicode_prefix
413+
@since(1.5)
414+
def unhex(col):
415+
"""Inverse of hex. Interprets each pair of characters as a hexadecimal number
416+
and converts to the byte representation of number.
417+
418+
>>> sqlContext.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect()
419+
[Row(unhex(a)=bytearray(b'ABC'))]
420+
"""
421+
sc = SparkContext._active_spark_context
422+
jc = sc._jvm.functions.unhex(_to_java_column(col))
423+
return Column(jc)
424+
425+
398426
@ignore_unicode_prefix
399427
@since(1.5)
400428
def sha1(col):

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala

Lines changed: 18 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -287,15 +287,18 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser {
287287
throw new AnalysisException(s"invalid function approximate($floatLit) $udfName")
288288
}
289289
}
290-
| CASE ~> expression.? ~ rep1(WHEN ~> expression ~ (THEN ~> expression)) ~
291-
(ELSE ~> expression).? <~ END ^^ {
292-
case casePart ~ altPart ~ elsePart =>
293-
val branches = altPart.flatMap { case whenExpr ~ thenExpr =>
294-
Seq(whenExpr, thenExpr)
295-
} ++ elsePart
296-
casePart.map(CaseKeyWhen(_, branches)).getOrElse(CaseWhen(branches))
297-
}
298-
)
290+
| CASE ~> whenThenElse ^^ CaseWhen
291+
| CASE ~> expression ~ whenThenElse ^^
292+
{ case keyPart ~ branches => CaseKeyWhen(keyPart, branches) }
293+
)
294+
295+
protected lazy val whenThenElse: Parser[List[Expression]] =
296+
rep1(WHEN ~> expression ~ (THEN ~> expression)) ~ (ELSE ~> expression).? <~ END ^^ {
297+
case altPart ~ elsePart =>
298+
altPart.flatMap { case whenExpr ~ thenExpr =>
299+
Seq(whenExpr, thenExpr)
300+
} ++ elsePart
301+
}
299302

300303
protected lazy val cast: Parser[Expression] =
301304
CAST ~ "(" ~> expression ~ (AS ~> dataType) <~ ")" ^^ {
@@ -354,6 +357,11 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser {
354357
protected lazy val signedPrimary: Parser[Expression] =
355358
sign ~ primary ^^ { case s ~ e => if (s == "-") UnaryMinus(e) else e}
356359

360+
protected lazy val attributeName: Parser[String] = acceptMatch("attribute name", {
361+
case lexical.Identifier(str) => str
362+
case lexical.Keyword(str) if !lexical.delimiters.contains(str) => str
363+
})
364+
357365
protected lazy val primary: PackratParser[Expression] =
358366
( literal
359367
| expression ~ ("[" ~> expression <~ "]") ^^
@@ -364,9 +372,9 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser {
364372
| "(" ~> expression <~ ")"
365373
| function
366374
| dotExpressionHeader
367-
| ident ^^ {case i => UnresolvedAttribute.quoted(i)}
368375
| signedPrimary
369376
| "~" ~> expression ^^ BitwiseNot
377+
| attributeName ^^ UnresolvedAttribute.quoted
370378
)
371379

372380
protected lazy val dotExpressionHeader: Parser[Expression] =

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -168,7 +168,7 @@ object FunctionRegistry {
168168
expression[Substring]("substring"),
169169
expression[UnBase64]("unbase64"),
170170
expression[Upper]("ucase"),
171-
expression[UnHex]("unhex"),
171+
expression[Unhex]("unhex"),
172172
expression[Upper]("upper"),
173173

174174
// datetime functions

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala

Lines changed: 45 additions & 38 deletions
Original file line numberDiff line numberDiff line change
@@ -298,6 +298,21 @@ case class Bin(child: Expression)
298298
}
299299
}
300300

301+
object Hex {
302+
val hexDigits = Array[Char](
303+
'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'
304+
).map(_.toByte)
305+
306+
// lookup table to translate '0' -> 0 ... 'F'/'f' -> 15
307+
val unhexDigits = {
308+
val array = Array.fill[Byte](128)(-1)
309+
(0 to 9).foreach(i => array('0' + i) = i.toByte)
310+
(0 to 5).foreach(i => array('A' + i) = (i + 10).toByte)
311+
(0 to 5).foreach(i => array('a' + i) = (i + 10).toByte)
312+
array
313+
}
314+
}
315+
301316
/**
302317
* If the argument is an INT or binary, hex returns the number as a STRING in hexadecimal format.
303318
* Otherwise if the number is a STRING, it converts each character into its hex representation
@@ -307,7 +322,7 @@ case class Hex(child: Expression) extends UnaryExpression with ExpectsInputTypes
307322
// TODO: Create code-gen version.
308323

309324
override def inputTypes: Seq[AbstractDataType] =
310-
Seq(TypeCollection(LongType, StringType, BinaryType))
325+
Seq(TypeCollection(LongType, BinaryType, StringType))
311326

312327
override def dataType: DataType = StringType
313328

@@ -319,30 +334,18 @@ case class Hex(child: Expression) extends UnaryExpression with ExpectsInputTypes
319334
child.dataType match {
320335
case LongType => hex(num.asInstanceOf[Long])
321336
case BinaryType => hex(num.asInstanceOf[Array[Byte]])
322-
case StringType => hex(num.asInstanceOf[UTF8String])
337+
case StringType => hex(num.asInstanceOf[UTF8String].getBytes)
323338
}
324339
}
325340
}
326341

327-
/**
328-
* Converts every character in s to two hex digits.
329-
*/
330-
private def hex(str: UTF8String): UTF8String = {
331-
hex(str.getBytes)
332-
}
333-
334-
private def hex(bytes: Array[Byte]): UTF8String = {
335-
doHex(bytes, bytes.length)
336-
}
337-
338-
private def doHex(bytes: Array[Byte], length: Int): UTF8String = {
342+
private[this] def hex(bytes: Array[Byte]): UTF8String = {
343+
val length = bytes.length
339344
val value = new Array[Byte](length * 2)
340345
var i = 0
341346
while (i < length) {
342-
value(i * 2) = Character.toUpperCase(Character.forDigit(
343-
(bytes(i) & 0xF0) >>> 4, 16)).toByte
344-
value(i * 2 + 1) = Character.toUpperCase(Character.forDigit(
345-
bytes(i) & 0x0F, 16)).toByte
347+
value(i * 2) = Hex.hexDigits((bytes(i) & 0xF0) >> 4)
348+
value(i * 2 + 1) = Hex.hexDigits(bytes(i) & 0x0F)
346349
i += 1
347350
}
348351
UTF8String.fromBytes(value)
@@ -355,24 +358,23 @@ case class Hex(child: Expression) extends UnaryExpression with ExpectsInputTypes
355358
var len = 0
356359
do {
357360
len += 1
358-
value(value.length - len) =
359-
Character.toUpperCase(Character.forDigit((numBuf & 0xF).toInt, 16)).toByte
361+
value(value.length - len) = Hex.hexDigits((numBuf & 0xF).toInt)
360362
numBuf >>>= 4
361363
} while (numBuf != 0)
362364
UTF8String.fromBytes(java.util.Arrays.copyOfRange(value, value.length - len, value.length))
363365
}
364366
}
365367

366-
367368
/**
368369
* Performs the inverse operation of HEX.
369370
* Resulting characters are returned as a byte array.
370371
*/
371-
case class UnHex(child: Expression) extends UnaryExpression with ExpectsInputTypes {
372+
case class Unhex(child: Expression) extends UnaryExpression with ExpectsInputTypes {
372373
// TODO: Create code-gen version.
373374

374375
override def inputTypes: Seq[AbstractDataType] = Seq(StringType)
375376

377+
override def nullable: Boolean = true
376378
override def dataType: DataType = BinaryType
377379

378380
override def eval(input: InternalRow): Any = {
@@ -384,26 +386,31 @@ case class UnHex(child: Expression) extends UnaryExpression with ExpectsInputTyp
384386
}
385387
}
386388

387-
private val unhexDigits = {
388-
val array = Array.fill[Byte](128)(-1)
389-
(0 to 9).foreach(i => array('0' + i) = i.toByte)
390-
(0 to 5).foreach(i => array('A' + i) = (i + 10).toByte)
391-
(0 to 5).foreach(i => array('a' + i) = (i + 10).toByte)
392-
array
393-
}
394-
395-
private def unhex(inputBytes: Array[Byte]): Array[Byte] = {
396-
var bytes = inputBytes
389+
private[this] def unhex(bytes: Array[Byte]): Array[Byte] = {
390+
val out = new Array[Byte]((bytes.length + 1) >> 1)
391+
var i = 0
397392
if ((bytes.length & 0x01) != 0) {
398-
bytes = '0'.toByte +: bytes
393+
// padding with '0'
394+
if (bytes(0) < 0) {
395+
return null
396+
}
397+
val v = Hex.unhexDigits(bytes(0))
398+
if (v == -1) {
399+
return null
400+
}
401+
out(0) = v
402+
i += 1
399403
}
400-
val out = new Array[Byte](bytes.length >> 1)
401404
// two characters form the hex value.
402-
var i = 0
403405
while (i < bytes.length) {
404-
val first = unhexDigits(bytes(i))
405-
val second = unhexDigits(bytes(i + 1))
406-
if (first == -1 || second == -1) { return null}
406+
if (bytes(i) < 0 || bytes(i + 1) < 0) {
407+
return null
408+
}
409+
val first = Hex.unhexDigits(bytes(i))
410+
val second = Hex.unhexDigits(bytes(i + 1))
411+
if (first == -1 || second == -1) {
412+
return null
413+
}
407414
out(i / 2) = (((first << 4) | second) & 0xFF).toByte
408415
i += 2
409416
}

0 commit comments

Comments
 (0)