Skip to content

Commit 42d47a3

Browse files
committed
Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
2 parents c3f4a02 + efc452a commit 42d47a3

File tree

11 files changed

+173
-73
lines changed

11 files changed

+173
-73
lines changed

python/pyspark/cloudpickle.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -243,10 +243,10 @@ def save_function(self, obj, name=None, pack=struct.pack):
243243
# if func is lambda, def'ed at prompt, is in main, or is nested, then
244244
# we'll pickle the actual function object rather than simply saving a
245245
# reference (as is done in default pickler), via save_function_tuple.
246-
if islambda(obj) or obj.func_code.co_filename == '<stdin>' or themodule == None:
246+
if islambda(obj) or obj.func_code.co_filename == '<stdin>' or themodule is None:
247247
#Force server to import modules that have been imported in main
248248
modList = None
249-
if themodule == None and not self.savedForceImports:
249+
if themodule is None and not self.savedForceImports:
250250
mainmod = sys.modules['__main__']
251251
if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'):
252252
modList = list(mainmod.___pyc_forcedImports__)

python/pyspark/conf.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@
3030
u'local'
3131
>>> sc.appName
3232
u'My app'
33-
>>> sc.sparkHome == None
33+
>>> sc.sparkHome is None
3434
True
3535
3636
>>> conf = SparkConf(loadDefaults=False)
@@ -116,7 +116,7 @@ def setSparkHome(self, value):
116116

117117
def setExecutorEnv(self, key=None, value=None, pairs=None):
118118
"""Set an environment variable to be passed to executors."""
119-
if (key != None and pairs != None) or (key == None and pairs == None):
119+
if (key is not None and pairs is not None) or (key is None and pairs is None):
120120
raise Exception("Either pass one key-value pair or a list of pairs")
121121
elif key != None:
122122
self._jconf.setExecutorEnv(key, value)

python/pyspark/rddsampler.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,7 @@ def getPoissonSample(self, split, mean):
8282
return (num_arrivals - 1)
8383

8484
def shuffle(self, vals):
85-
if self._random == None:
85+
if self._random is None:
8686
self.initRandomGenerator(0) # this should only ever called on the master so
8787
# the split does not matter
8888

python/pyspark/shell.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@
3535
from pyspark.storagelevel import StorageLevel
3636

3737
# this is the equivalent of ADD_JARS
38-
add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None
38+
add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") is not None else None
3939

4040
if os.environ.get("SPARK_EXECUTOR_URI"):
4141
SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"])
@@ -55,7 +55,7 @@
5555
platform.python_build()[1]))
5656
print("SparkContext available as sc.")
5757

58-
if add_files != None:
58+
if add_files is not None:
5959
print("Adding files: [%s]" % ", ".join(add_files))
6060

6161
# The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,

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

Lines changed: 12 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -215,19 +215,19 @@ case class EndsWith(left: Expression, right: Expression)
215215
case class Substring(str: Expression, pos: Expression, len: Expression) extends Expression {
216216

217217
type EvaluatedType = Any
218-
219-
def nullable: Boolean = true
218+
219+
def nullable: Boolean = str.nullable || pos.nullable || len.nullable
220220
def dataType: DataType = {
221221
if (!resolved) {
222222
throw new UnresolvedException(this, s"Cannot resolve since $children are not resolved")
223223
}
224224
if (str.dataType == BinaryType) str.dataType else StringType
225225
}
226-
226+
227227
def references = children.flatMap(_.references).toSet
228-
228+
229229
override def children = str :: pos :: len :: Nil
230-
230+
231231
@inline
232232
def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int)
233233
(implicit ev: (C=>IndexedSeqOptimized[T,_])): Any = {
@@ -237,40 +237,40 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends
237237
// refers to element i-1 in the sequence. If a start index i is less than 0, it refers
238238
// to the -ith element before the end of the sequence. If a start index i is 0, it
239239
// refers to the first element.
240-
240+
241241
val start = startPos match {
242242
case pos if pos > 0 => pos - 1
243243
case neg if neg < 0 => len + neg
244244
case _ => 0
245245
}
246-
246+
247247
val end = sliceLen match {
248248
case max if max == Integer.MAX_VALUE => max
249249
case x => start + x
250250
}
251-
251+
252252
str.slice(start, end)
253253
}
254-
254+
255255
override def eval(input: Row): Any = {
256256
val string = str.eval(input)
257257

258258
val po = pos.eval(input)
259259
val ln = len.eval(input)
260-
260+
261261
if ((string == null) || (po == null) || (ln == null)) {
262262
null
263263
} else {
264264
val start = po.asInstanceOf[Int]
265265
val length = ln.asInstanceOf[Int]
266-
266+
267267
string match {
268268
case ba: Array[Byte] => slice(ba, start, length)
269269
case other => slice(other.toString, start, length)
270270
}
271271
}
272272
}
273-
273+
274274
override def toString = len match {
275275
case max if max == Integer.MAX_VALUE => s"SUBSTR($str, $pos)"
276276
case _ => s"SUBSTR($str, $pos, $len)"

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -171,6 +171,9 @@ object NullPropagation extends Rule[LogicalPlan] {
171171
case Literal(candidate, _) if candidate == v => true
172172
case _ => false
173173
})) => Literal(true, BooleanType)
174+
case e @ Substring(Literal(null, _), _, _) => Literal(null, e.dataType)
175+
case e @ Substring(_, Literal(null, _), _) => Literal(null, e.dataType)
176+
case e @ Substring(_, _, Literal(null, _)) => Literal(null, e.dataType)
174177
// Put exceptional cases above if any
175178
case e: BinaryArithmetic => e.children match {
176179
case Literal(null, _) :: right :: Nil => Literal(null, e.dataType)

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -469,9 +469,9 @@ class ExpressionEvaluationSuite extends FunSuite {
469469

470470
test("Substring") {
471471
val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte)))
472-
472+
473473
val s = 'a.string.at(0)
474-
474+
475475
// substring from zero position with less-than-full length
476476
checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)), "ex", row)
477477
checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(2, IntegerType)), "ex", row)
@@ -501,7 +501,7 @@ class ExpressionEvaluationSuite extends FunSuite {
501501

502502
// substring(null, _, _) -> null
503503
checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), null, new GenericRow(Array[Any](null)))
504-
504+
505505
// substring(_, null, _) -> null
506506
checkEvaluation(Substring(s, Literal(null, IntegerType), Literal(4, IntegerType)), null, row)
507507

@@ -514,6 +514,12 @@ class ExpressionEvaluationSuite extends FunSuite {
514514

515515
// 2-arg substring from nonzero position
516516
checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "xample", row)
517+
518+
val s_notNull = 'a.string.notNull.at(0)
519+
520+
assert(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === true)
521+
assert(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === false)
522+
assert(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable === true)
523+
assert(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable === true)
517524
}
518525
}
519-

sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@
1717

1818
package org.apache.spark.sql.api.java
1919

20+
import java.util.{List => JList}
21+
2022
import org.apache.spark.Partitioner
2123
import org.apache.spark.api.java.{JavaRDDLike, JavaRDD}
2224
import org.apache.spark.api.java.function.{Function => JFunction}
@@ -96,6 +98,20 @@ class JavaSchemaRDD(
9698
this
9799
}
98100

101+
// Overridden actions from JavaRDDLike.
102+
103+
override def collect(): JList[Row] = {
104+
import scala.collection.JavaConversions._
105+
val arr: java.util.Collection[Row] = baseSchemaRDD.collect().toSeq.map(new Row(_))
106+
new java.util.ArrayList(arr)
107+
}
108+
109+
override def take(num: Int): JList[Row] = {
110+
import scala.collection.JavaConversions._
111+
val arr: java.util.Collection[Row] = baseSchemaRDD.take(num).toSeq.map(new Row(_))
112+
new java.util.ArrayList(arr)
113+
}
114+
99115
// Transformations (return a new RDD)
100116

101117
/**

sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala

Lines changed: 91 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -17,27 +17,34 @@
1717

1818
package org.apache.spark.sql.parquet
1919

20+
import scala.collection.JavaConversions._
21+
import scala.collection.mutable
22+
import scala.util.Try
23+
2024
import java.io.IOException
25+
import java.lang.{Long => JLong}
2126
import java.text.SimpleDateFormat
22-
import java.util.Date
27+
import java.util.{Date, List => JList}
2328

2429
import org.apache.hadoop.conf.Configuration
25-
import org.apache.hadoop.fs.Path
30+
import org.apache.hadoop.fs.{FileStatus, Path}
2631
import org.apache.hadoop.mapreduce._
2732
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
28-
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat, FileOutputCommitter}
33+
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
34+
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
2935

30-
import parquet.hadoop.{ParquetRecordReader, ParquetInputFormat, ParquetOutputFormat}
31-
import parquet.hadoop.api.ReadSupport
36+
import parquet.hadoop._
37+
import parquet.hadoop.api.{InitContext, ReadSupport}
38+
import parquet.hadoop.metadata.GlobalMetaData
3239
import parquet.hadoop.util.ContextUtil
33-
import parquet.io.InvalidRecordException
40+
import parquet.io.ParquetDecodingException
3441
import parquet.schema.MessageType
3542

36-
import org.apache.spark.{Logging, SerializableWritable, TaskContext}
3743
import org.apache.spark.rdd.RDD
3844
import org.apache.spark.sql.SQLContext
3945
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row}
4046
import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode}
47+
import org.apache.spark.{Logging, SerializableWritable, TaskContext}
4148

4249
/**
4350
* Parquet table scan operator. Imports the file that backs the given
@@ -55,16 +62,14 @@ case class ParquetTableScan(
5562
override def execute(): RDD[Row] = {
5663
val sc = sqlContext.sparkContext
5764
val job = new Job(sc.hadoopConfiguration)
58-
ParquetInputFormat.setReadSupportClass(
59-
job,
60-
classOf[org.apache.spark.sql.parquet.RowReadSupport])
65+
ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport])
66+
6167
val conf: Configuration = ContextUtil.getConfiguration(job)
62-
val fileList = FileSystemHelper.listFiles(relation.path, conf)
63-
// add all paths in the directory but skip "hidden" ones such
64-
// as "_SUCCESS" and "_metadata"
65-
for (path <- fileList if !path.getName.startsWith("_")) {
66-
NewFileInputFormat.addInputPath(job, path)
68+
val qualifiedPath = {
69+
val path = new Path(relation.path)
70+
path.getFileSystem(conf).makeQualified(path)
6771
}
72+
NewFileInputFormat.addInputPath(job, qualifiedPath)
6873

6974
// Store both requested and original schema in `Configuration`
7075
conf.set(
@@ -87,7 +92,7 @@ case class ParquetTableScan(
8792

8893
sc.newAPIHadoopRDD(
8994
conf,
90-
classOf[org.apache.spark.sql.parquet.FilteringParquetRowInputFormat],
95+
classOf[FilteringParquetRowInputFormat],
9196
classOf[Void],
9297
classOf[Row])
9398
.map(_._2)
@@ -122,14 +127,7 @@ case class ParquetTableScan(
122127
private def validateProjection(projection: Seq[Attribute]): Boolean = {
123128
val original: MessageType = relation.parquetSchema
124129
val candidate: MessageType = ParquetTypesConverter.convertFromAttributes(projection)
125-
try {
126-
original.checkContains(candidate)
127-
true
128-
} catch {
129-
case e: InvalidRecordException => {
130-
false
131-
}
132-
}
130+
Try(original.checkContains(candidate)).isSuccess
133131
}
134132
}
135133

@@ -302,6 +300,11 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int)
302300
*/
303301
private[parquet] class FilteringParquetRowInputFormat
304302
extends parquet.hadoop.ParquetInputFormat[Row] with Logging {
303+
304+
private var footers: JList[Footer] = _
305+
306+
private var fileStatuses= Map.empty[Path, FileStatus]
307+
305308
override def createRecordReader(
306309
inputSplit: InputSplit,
307310
taskAttemptContext: TaskAttemptContext): RecordReader[Void, Row] = {
@@ -318,6 +321,70 @@ private[parquet] class FilteringParquetRowInputFormat
318321
new ParquetRecordReader[Row](readSupport)
319322
}
320323
}
324+
325+
override def getFooters(jobContext: JobContext): JList[Footer] = {
326+
if (footers eq null) {
327+
val statuses = listStatus(jobContext)
328+
fileStatuses = statuses.map(file => file.getPath -> file).toMap
329+
footers = getFooters(ContextUtil.getConfiguration(jobContext), statuses)
330+
}
331+
332+
footers
333+
}
334+
335+
// TODO Remove this method and related code once PARQUET-16 is fixed
336+
// This method together with the `getFooters` method and the `fileStatuses` field are just used
337+
// to mimic this PR: https://github.com/apache/incubator-parquet-mr/pull/17
338+
override def getSplits(
339+
configuration: Configuration,
340+
footers: JList[Footer]): JList[ParquetInputSplit] = {
341+
342+
val maxSplitSize: JLong = configuration.getLong("mapred.max.split.size", Long.MaxValue)
343+
val minSplitSize: JLong =
344+
Math.max(getFormatMinSplitSize(), configuration.getLong("mapred.min.split.size", 0L))
345+
if (maxSplitSize < 0 || minSplitSize < 0) {
346+
throw new ParquetDecodingException(
347+
s"maxSplitSize or minSplitSie should not be negative: maxSplitSize = $maxSplitSize;" +
348+
s" minSplitSize = $minSplitSize")
349+
}
350+
351+
val getGlobalMetaData =
352+
classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]])
353+
getGlobalMetaData.setAccessible(true)
354+
val globalMetaData = getGlobalMetaData.invoke(null, footers).asInstanceOf[GlobalMetaData]
355+
356+
val readContext = getReadSupport(configuration).init(
357+
new InitContext(configuration,
358+
globalMetaData.getKeyValueMetaData(),
359+
globalMetaData.getSchema()))
360+
361+
val generateSplits =
362+
classOf[ParquetInputFormat[_]].getDeclaredMethods.find(_.getName == "generateSplits").get
363+
generateSplits.setAccessible(true)
364+
365+
val splits = mutable.ArrayBuffer.empty[ParquetInputSplit]
366+
for (footer <- footers) {
367+
val fs = footer.getFile.getFileSystem(configuration)
368+
val file = footer.getFile
369+
val fileStatus = fileStatuses.getOrElse(file, fs.getFileStatus(file))
370+
val parquetMetaData = footer.getParquetMetadata
371+
val blocks = parquetMetaData.getBlocks
372+
val fileBlockLocations = fs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen)
373+
splits.addAll(
374+
generateSplits.invoke(
375+
null,
376+
blocks,
377+
fileBlockLocations,
378+
fileStatus,
379+
parquetMetaData.getFileMetaData,
380+
readContext.getRequestedSchema.toString,
381+
readContext.getReadSupportMetadata,
382+
minSplitSize,
383+
maxSplitSize).asInstanceOf[JList[ParquetInputSplit]])
384+
}
385+
386+
splits
387+
}
321388
}
322389

323390
private[parquet] object FileSystemHelper {

0 commit comments

Comments
 (0)