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
56 changes: 56 additions & 0 deletions python/pyspark/sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -187,6 +187,56 @@ def func(split, iterator):
jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd())
return SchemaRDD(jschema_rdd, self)

def csvFile(self, path, delimiter = ",", quote = "\"", header = False):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you could use '"' here without escape

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also no space between name of argument and default value, such as header=False

"""
Loads a CSV file (according to RFC 4180) and returns the result as a L{SchemaRDD}.
header flag specified if first line of each file should be treated as header.

NOTE: If there are new line characters inside quoted fields this method may fail to
parse correctly, because the two lines may be in different partitions. Use
L{SQLContext#csvRDD} to parse such files.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SQLContext.csvRDD


>>> import tempfile, shutil
>>> csvFile = tempfile.mkdtemp()
>>> shutil.rmtree(csvFile)
>>> ofn = open(csvFile, 'w')
>>> for csvStr in csvStrings:
... print>>ofn, csvStr
>>> ofn.close()
>>> csv = sqlCtx.csvFile(csvFile, delimiter = ", ", header = True)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

no space here: header=True

>>> sqlCtx.registerRDDAsTable(csv, "csvTable")
>>> csvRes = sqlCtx.sql("SELECT Year FROM csvTable WHERE Make = 'Ford'")
>>> csvRes.collect()
[{u'Year': u'1997'}]
"""
jschema_rdd = self._ssql_ctx.csvFile(path, delimiter, quote, header)
return SchemaRDD(jschema_rdd, self)

def csvRDD(self, rdd, delimiter = ",", quote = "\"", header = False):
"""
Parses an RDD of String as a CSV (according to RFC 4180) and returns the result as a
L{SchemaRDD}.

NOTE: If there are new line characters inside quoted fields, use wholeTextFile to
read each file into a single partition.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

PS: wholeTextFile() will read a file as single string, may it's not what you want. Also, it will return a (path, content) pair.


>>> csvrdd = sqlCtx.csvRDD(csv, delimiter = ", ", header = True)
>>> sqlCtx.registerRDDAsTable(csvrdd, "csvTable2")
>>> csvRes = sqlCtx.sql("SELECT count(*) FROM csvTable2")
>>> csvRes.collect() == [{"c0": 3}]
True
"""
def func(split, iterator):
for x in iterator:
if not isinstance(x, basestring):
x = unicode(x)
yield x.encode("utf-8")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

x could be str (in "utf-8"), see saveAsTextFile()

keyed = PipelinedRDD(rdd, func)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

PipelinedRDD is an implementation detail, so it's better to use rdd.mapPartitionsWithIndex() here (mapPartitions() will be better).

keyed._bypass_serializer = True
jrdd = keyed._jrdd.map(self._jvm.BytesToString())
jschema_rdd = self._ssql_ctx.csvRDD(jrdd.rdd(), delimiter, quote, header)
return SchemaRDD(jschema_rdd, self)

def sql(self, sqlQuery):
"""Return a L{SchemaRDD} representing the result of the given query.

Expand Down Expand Up @@ -505,6 +555,12 @@ def _test():
]
globs['jsonStrings'] = jsonStrings
globs['json'] = sc.parallelize(jsonStrings)
csvStrings = ['Year, Make, Model, Description',
'"1997", "Ford", "E350", ',
'2000, Mercury, "Cougar", "Really ""Good"" car"',
'2007, Honda, "Civic", ']
globs['csvStrings'] = csvStrings
globs['csv'] = sc.parallelize(csvStrings)
globs['nestedRdd1'] = sc.parallelize([
{"f1": array('i', [1, 2]), "f2": {"row1": 1.0}},
{"f1": array('i', [2, 3]), "f2": {"row2": 2.0}}])
Expand Down
95 changes: 95 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.optimizer.Optimizer
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.columnar.InMemoryRelation
import org.apache.spark.sql.csv._
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.SparkStrategies
import org.apache.spark.sql.json._
Expand Down Expand Up @@ -129,6 +130,100 @@ class SQLContext(@transient val sparkContext: SparkContext)
def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD =
new SchemaRDD(this, JsonRDD.inferSchema(json, samplingRatio))

/**
* Loads a CSV file (according to RFC 4180) and returns the result as a [[SchemaRDD]].
*
* NOTE: If there are new line characters inside quoted fields this method may fail to
* parse correctly, because the two lines may be in different partitions. Use
* [[SQLContext#csvRDD]] to parse such files.
*
* @param path path to input file
* @param schema StructType object to specify schema (field names and types). This will
* override field names if header is used
* @param delimiter Optional delimiter (default is comma)
* @param quote Optional quote character or string (default is '"')
* @param header Optional flag to indicate first line of each file is the header
* (default is false)
*/
def csvFile(
path: String,
schema: StructType,
delimiter: String,
quote: Char,
header: Boolean): SchemaRDD = {
val csv = sparkContext.textFile(path)
csvRDD(csv, schema, delimiter, quote, header)
}

/**
* Loads a CSV file (according to RFC 4180) and returns the result as a [[SchemaRDD]].
* It infers the schema based on the first record.
*
* NOTE: If there are new line characters inside quoted fields this method may fail to
* parse correctly, because the two lines may be in different partitions. Use
* [[SQLContext#csvRDD]] to parse such files.
*
* @param path path to input file
* @param delimiter Optional delimiter (default is comma)
* @param quote Optional quote character or string (default is '"')
* @param header Optional flag to indicate first line of each file is the header
* (default is false)
*/
def csvFile(
path: String,
delimiter: String = ",",
quote: Char = '"',
header: Boolean = false): SchemaRDD = {
val csv = sparkContext.textFile(path)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

textFile creates an RDD of lines. This actually doesn't work if a record contains new lines (inside quotes).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To do this properly, we would either need a new input format that handles CSV line splits, or assemble the lines back from textFile.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

CsvTokenizer takes an Iterator[String] and implements Iterator[Array[Any]]. It's next() may end up reading two or more lines if it needs to (e.g., when a quoted field spans multiple lines). This assumes quoted fields are not split between partitions (noted in API documentation).

csvRDD(csv, delimiter, quote, header)
}


/**
* Parses an RDD of String as a CSV (according to RFC 4180) and returns the result as a
* [[SchemaRDD]].
*
* NOTE: If there are new line characters inside quoted fields, use
* [[SparkContext#wholeTextFiles]] to read each file into a single partition.
*
* @param csv input RDD
* @param schema StructType object to specify schema (field names and types). This will
* override field names if header is used
* @param delimiter Optional delimiter (default is comma)
* @param quote Optional quote character of strig (default is '"')
* @param header Optional flag to indicate first line of each file is the hader
* (default is false)
*/
def csvRDD(
csv: RDD[String],
schema: StructType,
delimiter: String,
quote: Char,
header: Boolean): SchemaRDD = {
new SchemaRDD(this, CsvRDD.inferSchema(csv, delimiter, quote, Some(schema), header))
}

/**
* Parses an RDD of String as a CSV (according to RFC 4180) and returns the result as a
* [[SchemaRDD]]. It infers the schema based on the first record.
*
* NOTE: If there are new line characters inside quoted fields, use
* [[SparkContext#wholeTextFiles]] to read each file into a single partition.
*
* @param csv input RDD
* @param delimiter Optional delimiter (default is comma)
* @param quote Optional quote character of strig (default is '"')
* @param header Optional flag to indicate first line of each file is the hader
* (default is false)
*/
def csvRDD(
csv: RDD[String],
delimiter: String = ",",
quote: Char = '"',
header: Boolean = false): SchemaRDD = {
new SchemaRDD(this, CsvRDD.inferSchema(csv, delimiter, quote, None, header))
}

/**
* :: Experimental ::
* Creates an empty parquet file with the schema of class `A`, which can be registered as a table.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration

import org.apache.spark.annotation.Experimental
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.sql.csv.CsvRDD
import org.apache.spark.sql.json.JsonRDD
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow}
Expand Down Expand Up @@ -122,6 +123,100 @@ class JavaSQLContext(val sqlContext: SQLContext) {
def jsonRDD(json: JavaRDD[String]): JavaSchemaRDD =
new JavaSchemaRDD(sqlContext, JsonRDD.inferSchema(json, 1.0))

/**
* Loads a CSV file (according to RFC 4180) and returns the result as a [[JavaSchemaRDD]].
*
* NOTE: If there are new line characters inside quoted fields this method may fail to
* parse correctly, because the two lines may be in different partitions. Use
* [[SQLContext#csvRDD]] to parse such files.
*
* @param path path to input file
* @param schema StructType object to specify schema (field names and types). This will
* override field names if header is used
* @param delimiter Optional delimiter (default is comma)
* @param quote Optional quote character or string (default is '"')
* @param header Optional flag to indicate first line of each file is the header
* (default is false)
*/
def csvFile(
path: String,
schema: StructType,
delimiter: String,
quote: Char,
header: Boolean): JavaSchemaRDD = {
val csv = sqlContext.sparkContext.textFile(path)
csvRDD(csv, schema, delimiter, quote, header)
}

/**
* Loads a CSV file (according to RFC 4180) and returns the result as a [[JavaSchemaRDD]].
* It infers the schema based on the first record.
*
* NOTE: If there are new line characters inside quoted fields this method may fail to
* parse correctly, because the two lines may be in different partitions. Use
* [[SQLContext#csvRDD]] to parse such files.
*
* @param path path to input file
* @param delimiter Optional delimiter (default is comma)
* @param quote Optional quote character or string (default is '"')
* @param header Optional flag to indicate first line of each file is the header
* (default is false)
*/
def csvFile(
path: String,
delimiter: String,
quote: Char,
header: Boolean): JavaSchemaRDD = {
val csv = sqlContext.sparkContext.textFile(path)
csvRDD(csv, delimiter, quote, header)
}

/**
* Parses an RDD of String as a CSV (according to RFC 4180) and returns the result as a
* [[JavaSchemaRDD]].
*
* NOTE: If there are new line characters inside quoted fields, use
* [[JavaSparkContext#wholeTextFiles]] to read each file into a single partition.
*
* @param csv input RDD
* @param schema optional StructType object to specify schema (field names and types). This will
* override field names if header is used
* @param delimiter Optional delimiter (default is comma)
* @param quote Optional quote character of strig (default is '"')
* @param header Optional flag to indicate first line of each file is the hader
* (default is false)
*/
def csvRDD(
csv: JavaRDD[String],
schema: StructType,
delimiter: String,
quote: Char,
header: Boolean): JavaSchemaRDD = {
new JavaSchemaRDD(sqlContext, CsvRDD.inferSchema(csv, delimiter, quote, Some(schema), header))
}

/**
* Parses an RDD of String as a CSV (according to RFC 4180) and returns the result as a
* [[JavaSchemaRDD]]. It infers the schema based on the first record.
*
* NOTE: If there are new line characters inside quoted fields, use
* [[JavaSparkContext#wholeTextFiles]] to read each file into a single partition.
*
* @param csv input RDD
* @param delimiter Optional delimiter (default is comma)
* @param quote Optional quote character of strig (default is '"')
* @param header Optional flag to indicate first line of each file is the hader
* (default is false)
*/
def csvRDD(
csv: JavaRDD[String],
delimiter: String,
quote: Char,
header: Boolean): JavaSchemaRDD = {
new JavaSchemaRDD(sqlContext, CsvRDD.inferSchema(csv, delimiter, quote, None, header))
}


/**
* Registers the given RDD as a temporary table in the catalog. Temporary tables exist only
* during the lifetime of this instance of SQLContext.
Expand Down
107 changes: 107 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/csv/CsvRDD.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
/*
* 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.csv

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan}
import org.apache.spark.sql.Logging

private[sql] object CsvRDD extends Logging {

/**
* Infers schema of a CSV file. It uses the first row of the first partition to
* infer number of columns. If header flag is set, all lines that equal the first line
* are filtered before parsing.
*
* o If a line contains fewer tokens than the schema, it is padded with nulls
* o If a line has more tokens than the schema, extra tokens are ignored.
*/
private[sql] def inferSchema(
csv: RDD[String],
delimiter: String,
quote: Char,
userSchema: Option[StructType],
useHeader: Boolean): LogicalPlan = {

val firstLine = csv.first()
val schema = userSchema match {
case Some(userSupportedSchema) => userSupportedSchema
case None =>
// Assume first row is representative and use it to determine number of fields
val firstRow = new CsvTokenizer(Seq(firstLine).iterator, delimiter, quote).next()
val header = if (useHeader) {
logger.info(s"Using header line: $firstLine")
firstRow
} else {
firstRow.zipWithIndex.map { case (value, index) => s"V$index"}
}
// By default fields are assumed to be StringType
val schemaFields = header.map { fieldName =>
StructField(fieldName, StringType, nullable = true)
}
StructType(schemaFields)
}

val numFields = schema.fields.length
logger.info(s"Parsing CSV with $numFields.")
val row = new GenericMutableRow(numFields)
val projection = schemaCaster(asAttributes(schema))

val parsedCSV = csv.mapPartitions { iter =>
// When using header, any input line that equals firstLine is assumed to be header
val csvIter = if (useHeader) {
iter.filter(_ != firstLine)
} else {
iter
}
val tokenIter = new CsvTokenizer(csvIter, delimiter, quote)
parseCSV(tokenIter, schema.fields, projection, row)
}

SparkLogicalPlan(ExistingRdd(asAttributes(schema), parsedCSV))
}

protected def schemaCaster(schema: Seq[AttributeReference]): MutableProjection = {
val startSchema = (1 to schema.length).toSeq.map(
index => new AttributeReference(s"V$index", StringType, nullable = true)())
val casts = schema.zipWithIndex.map { case (ar, i) => Cast(startSchema(i), ar.dataType) }
new MutableProjection(casts, startSchema)
}

private def parseCSV(
iter: Iterator[Array[String]],
schemaFields: Seq[StructField],
projection: MutableProjection,
row: GenericMutableRow): Iterator[Row] = {
iter.map { tokens =>
schemaFields.zipWithIndex.foreach {
case (StructField(name, dataType, _), index) =>
row.update(index, tokens(index))
}
projection(row)
}
}

private def asAttributes(struct: StructType): Seq[AttributeReference] = {
struct.fields.map(field => AttributeReference(field.name, field.dataType, nullable = true)())
}
}

Loading