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
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.sql.catalyst.expressions

import java.net.URLDecoder
import java.sql.{Date, Timestamp}
import java.text.{DateFormat, SimpleDateFormat}

Expand Down Expand Up @@ -158,6 +159,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
if (periodIdx != -1 && n.length() - periodIdx > 9) {
n = n.substring(0, periodIdx + 10)
}
// Timestamp value in the partition could be in the form: (2015-02-09 00%3A55%3A00)
n = URLDecoder.decode(n, "UTF-8");
try Timestamp.valueOf(n) catch { case _: java.lang.IllegalArgumentException => null }
})
case BooleanType =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -434,7 +434,7 @@ case object BooleanType extends BooleanType
* @group dataType
*/
@DeveloperApi
class TimestampType private() extends NativeType {
class TimestampType private() extends NativeType with PrimitiveType {
// The companion object and this class is separated so the companion object also subclasses
// this type. Otherwise, the companion object would be of type "TimestampType$" in byte code.
// Defined with a private constructor so the companion object is the only possible instantiation.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,8 @@ private[sql] case class ParquetRelation(
path: String,
@transient conf: Option[Configuration],
@transient sqlContext: SQLContext,
partitioningAttributes: Seq[Attribute] = Nil)
partitioningAttributes: Seq[Attribute] = Nil,
partitionValues: String = "")
extends LeafNode with MultiInstanceRelation {

self: Product =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,18 +80,34 @@ private[sql] case class ParquetTableScan(
override def execute(): RDD[Row] = {
import parquet.filter2.compat.FilterCompat.FilterPredicateCompat

var partMap = mutable.HashMap[String, String]()
val sc = sqlContext.sparkContext
val job = new Job(sc.hadoopConfiguration)
ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport])

val conf: Configuration = ContextUtil.getConfiguration(job)

relation.path.split(",").foreach { curPath =>
val qualifiedPath = {
val path = new Path(curPath)
path.getFileSystem(conf).makeQualified(path)
if (requestedPartitionOrdinals.nonEmpty) {
val partVals = relation.partitionValues.split(",")
var i = 0
relation.path.split(",").foreach { curPath =>
val partition = partVals.apply(i)
i += 1
val qualifiedPath = {
val path = new Path(curPath)
path.getFileSystem(conf).makeQualified(path)
}
partMap += curPath->partition;
NewFileInputFormat.addInputPath(job, qualifiedPath)
}
} else {
relation.path.split(",").foreach { curPath =>
val qualifiedPath = {
val path = new Path(curPath)
path.getFileSystem(conf).makeQualified(path)
}
NewFileInputFormat.addInputPath(job, qualifiedPath)
}
NewFileInputFormat.addInputPath(job, qualifiedPath)
}

// Store both requested and original schema in `Configuration`
Expand Down Expand Up @@ -135,15 +151,18 @@ private[sql] case class ParquetTableScan(

baseRDD.mapPartitionsWithInputSplit { case (split, iter) =>
val partValue = "([^=]+)=([^=]+)".r
val iSplit = split.asInstanceOf[parquet.hadoop.ParquetInputSplit]
.getPath
.toString
val partValues =
split.asInstanceOf[parquet.hadoop.ParquetInputSplit]
.getPath
.toString
.split("/")
.flatMap {
case partValue(key, value) => Some(key -> value)
case _ => None
}.toMap
partMap.get(
iSplit.splitAt(iSplit.lastIndexOf("/"))._1)
.get
.split("/")
.flatMap {
case partValue(key, value) => Some(key -> value)
case _ => None
}.toMap

// Convert the partitioning attributes into the correct types
val partitionRowValues =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,15 +60,16 @@ private[hive] trait HiveStrategies {
implicit class LogicalPlanHacks(s: DataFrame) {
def lowerCase: DataFrame = DataFrame(s.sqlContext, s.logicalPlan)

def addPartitioningAttributes(attrs: Seq[Attribute]): DataFrame = {
def addPartitioningAttributes(attrs: Seq[Attribute], partVals: String): DataFrame = {
// Don't add the partitioning key if its already present in the data.
if (attrs.map(_.name).toSet.subsetOf(s.logicalPlan.output.map(_.name).toSet)) {
s
} else {
DataFrame(
s.sqlContext,
s.logicalPlan transform {
case p: ParquetRelation => p.copy(partitioningAttributes = attrs)
case p: ParquetRelation => p.copy(partitioningAttributes = attrs,
partitionValues = partVals)
})
}
}
Expand Down Expand Up @@ -137,14 +138,15 @@ private[hive] trait HiveStrategies {
pruningCondition(inputData)
}

val partitionLocations = partitions.map(_.getLocation)
val partitionLocations = partitions.map(part => part.getLocation)
val partitionNames = partitions.map(part => part.getName)

if (partitionLocations.isEmpty) {
PhysicalRDD(plan.output, sparkContext.emptyRDD[Row]) :: Nil
} else {
hiveContext
.parquetFile(partitionLocations: _*)
.addPartitioningAttributes(relation.partitionKeys)
.addPartitioningAttributes(relation.partitionKeys, partitionNames.mkString(","))
.lowerCase
.where(unresolvedOtherPredicates)
.select(unresolvedProjection: _*)
Expand Down