From fb14a064e6e2e735bb385012d46403c4c35a6286 Mon Sep 17 00:00:00 2001 From: scwf Date: Mon, 29 Sep 2014 21:45:36 +0800 Subject: [PATCH 01/18] initial support orc in spark sql --- .../plans/logical/basicOperators.scala | 23 + sql/core/pom.xml | 15 + .../scala/org/apache/spark/sql/SQLConf.scala | 7 + .../org/apache/spark/sql/SQLContext.scala | 44 ++ .../org/apache/spark/sql/SchemaRDDLike.scala | 13 +- .../spark/sql/execution/SparkStrategies.scala | 22 + .../apache/spark/sql/orc/OrcRelation.scala | 263 +++++++++++ .../spark/sql/orc/OrcTableOperations.scala | 418 ++++++++++++++++++ 8 files changed, 804 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/orc/OrcRelation.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/orc/OrcTableOperations.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 391508279bb8..e6fd08529227 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -114,6 +114,22 @@ case class InsertIntoTable( } } +case class InsertIntoOrcTable( + table: LogicalPlan, + partition: Map[String, Option[String]], + child: LogicalPlan, + overwrite: Boolean) + extends LogicalPlan { + // The table being inserted into is a child for the purposes of transformations. + def children = table :: child :: Nil + def references = Set.empty + def output = child.output + + override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { + case (childAttr, tableAttr) => childAttr.dataType == tableAttr.dataType + } +} + case class CreateTableAsSelect( databaseName: Option[String], tableName: String, @@ -128,6 +144,13 @@ case class WriteToFile( override def output = child.output } +case class WriteToOrcFile( + path: String, + child: LogicalPlan) extends UnaryNode { + def references = Set.empty + def output = child.output +} + case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { override def output = child.output } diff --git a/sql/core/pom.xml b/sql/core/pom.xml index bd110218d34f..724d632c480b 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -53,6 +53,21 @@ test-jar test + + org.spark-project.hive + hive-exec + ${hive.version} + + + commons-logging + commons-logging + + + org.slf4j + slf4j-api + + + com.twitter parquet-column diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index f6f4cf3b80d4..1fca2f380e7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -35,6 +35,7 @@ private[spark] object SQLConf { val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec" + val ORC_COMPRESSION = "spark.sql.orc.compression.codec" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -83,6 +84,12 @@ private[sql] trait SQLConf { /** The compression codec for writing to a Parquetfile */ private[spark] def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION, "snappy") + /** The compression codec for writing to a Orcfile + * Note: only support zlib now since we use ```OrcOutputFormat.getRecordWriter``` ,which is not + * allowed to configure thr compression kind + */ + private[spark] def orcCompressionCodec: String = getConf(ORC_COMPRESSION, "zlib") + /** The number of rows that will be */ private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "1000").toInt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a42bedbe6c04..c1b6e44af4f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.execution.SparkStrategies import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.sql.orc.OrcRelation /** * :: AlphaComponent :: @@ -145,6 +146,14 @@ class SQLContext(@transient val sparkContext: SparkContext) def parquetFile(path: String): SchemaRDD = new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) + /** + * Loads a Orc file, returning the result as a [[SchemaRDD]]. + * + * @group userf + */ + def orcFile(path: String): SchemaRDD = + new SchemaRDD(this, orc.OrcRelation(path, Some(sparkContext.hadoopConfiguration), this)) + /** * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. * It goes through the entire dataset once to determine the schema. @@ -244,6 +253,40 @@ class SQLContext(@transient val sparkContext: SparkContext) path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) } + /** + * :: Experimental :: + * Creates an empty parquet file with the schema of class `A`, which can be registered as a table. + * This registered table can be used as the target of future `insertInto` operations. + * + * {{{ + * val sqlContext = new SQLContext(...) + * import sqlContext._ + * + * case class Person(name: String, age: Int) + * createOrcFile[Person]("path/to/file.orc").registerTempTable("people") + * sql("INSERT INTO people SELECT 'michael', 29") + * }}} + * + * @tparam A A case class type that describes the desired schema of the parquet file to be + * created. + * @param path The path where the directory containing parquet metadata should be created. + * Data inserted into this table will also be stored at this location. + * @param allowExisting When false, an exception will be thrown if this directory already exists. + * @param conf A Hadoop configuration object that can be used to specify options to the parquet + * output format. + * + * @group userf + */ + @Experimental + def createOrcFile[A <: Product : TypeTag]( + path: String, + allowExisting: Boolean = true, + conf: Configuration = new Configuration()): SchemaRDD = { + new SchemaRDD( + this, + OrcRelation.createEmpty(path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) + } + /** * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only * during the lifetime of this instance of SQLContext. @@ -328,6 +371,7 @@ class SQLContext(@transient val sparkContext: SparkContext) HashJoin :: InMemoryScans :: ParquetOperations :: + OrcOperations:: BasicOperators :: CartesianProduct :: BroadcastNestedLoopJoin :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index e52eeb3e1c47..4e6546332cda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql import org.apache.spark.annotation.{DeveloperApi, Experimental} -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation /** * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java) @@ -77,6 +77,17 @@ private[sql] trait SchemaRDDLike { sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd } + /** + * Saves the contents of this `SchemaRDD` as a orc file, preserving the schema. Files that + * are written out using this method can be read back in as a SchemaRDD using the `orcFile` + * function. + * + * @group schema + */ + def saveAsOrcFile(path: String): Unit = { + sqlContext.executePlan(WriteToOrcFile(path, logicalPlan)).toRdd + } + /** * Registers this RDD as a temporary table using the given name. The lifetime of this temporary * table is tied to the [[SQLContext]] that was used to create this SchemaRDD. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 45687d960404..0ab938461b7c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.parquet._ +import org.apache.spark.sql.orc.{OrcTableScan, InsertIntoOrcTable, OrcRelation} private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => @@ -237,6 +238,27 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } + object OrcOperations extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case logical.WriteToOrcFile(path, child) => + val relation = + OrcRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) + InsertIntoOrcTable(relation, planLater(child), overwrite=true) :: Nil + case logical.InsertIntoOrcTable(table: OrcRelation, partition, child, overwrite) => + InsertIntoOrcTable(table, planLater(child), overwrite) :: Nil + case PhysicalOperation(projectList, filters, relation: OrcRelation) => + // TODO: need to implement predict push down. + val prunePushedDownFilters = identity[Seq[Expression]] _ + pruneFilterProject( + projectList, + filters, + prunePushedDownFilters, + OrcTableScan(_, relation, None)) :: Nil + + case _ => Nil + } + } + object InMemoryScans extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(projectList, filters, mem: InMemoryRelation) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcRelation.scala new file mode 100644 index 000000000000..8b5156c0f376 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcRelation.scala @@ -0,0 +1,263 @@ +/* + * 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.orc + +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedException, MultiInstanceRelation} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.types._ +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.permission.FsAction +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector +import org.apache.hadoop.hive.ql.io.orc._ +import org.apache.hadoop.mapred.{FileInputFormat => NewFileInputFormat, JobConf} +import org.apache.hadoop.hive.ql.io.orc.OrcProto.Type.Kind +import org.apache.hadoop.mapreduce.Job +import parquet.hadoop.util.ContextUtil +import java.util.Properties +import java.io.IOException +import scala.collection.mutable +import org.apache.spark.sql.SQLContext + + +private[sql] case class OrcRelation( + path: String, + @transient conf: Option[Configuration], + @transient sqlContext: SQLContext, + partitioningAttributes: Seq[Attribute] = Nil) + extends LeafNode with MultiInstanceRelation { + self: Product => + + val prop: Properties = new Properties + + var rowClass: Class[_] = null + + val fieldIdCache: mutable.Map[String, Int] = new mutable.HashMap[String, Int] + + val fieldNameTypeCache: mutable.Map[String, String] = new mutable.HashMap[String, String] + + override val output = orcSchema + + def orcSchema: Seq[Attribute] = { + val origPath = new Path(path) + val reader = OrcFileOperator.readMetaData(origPath) + + if (null != reader) { + val inspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] + val fields = inspector.getAllStructFieldRefs + + if (fields.size() == 0) { + return Seq.empty + } + + val totalType = reader.getTypes.get(0) + val keys = totalType.getFieldNamesList + val types = totalType.getSubtypesList + log.info("field name is {}", keys) + log.info("types is {}", types) + + val colBuff = new StringBuilder + val typeBuff = new StringBuilder + for (i <- 0 until fields.size()) { + val fieldName = fields.get(i).getFieldName + val typeName = fields.get(i).getFieldObjectInspector.getTypeName + colBuff.append(fieldName) + fieldNameTypeCache.put(fieldName, typeName) + fieldIdCache.put(fieldName, i) + colBuff.append(",") + typeBuff.append(typeName) + typeBuff.append(":") + } + colBuff.setLength(colBuff.length - 1) + typeBuff.setLength(typeBuff.length - 1) + prop.setProperty("columns", colBuff.toString()) + prop.setProperty("columns.types", typeBuff.toString()) + val attributes = convertToAttributes(reader, keys, types) + attributes + } else { + Seq.empty + } + } + + def convertToAttributes( + reader: Reader, + keys: java.util.List[String], + types: java.util.List[Integer]): Seq[Attribute] = { + val range = 0.until(keys.size()) + range.map { + i => reader.getTypes.get(types.get(i)).getKind match { + case Kind.BOOLEAN => + new AttributeReference(keys.get(i), BooleanType, false)() + case Kind.STRING => + new AttributeReference(keys.get(i), StringType, true)() + case Kind.BYTE => + new AttributeReference(keys.get(i), ByteType, true)() + case Kind.SHORT => + new AttributeReference(keys.get(i), ShortType, true)() + case Kind.INT => + new AttributeReference(keys.get(i), IntegerType, true)() + case Kind.LONG => + new AttributeReference(keys.get(i), LongType, false)() + case Kind.FLOAT => + new AttributeReference(keys.get(i), FloatType, false)() + case Kind.DOUBLE => + new AttributeReference(keys.get(i), DoubleType, false)() + case _ => { + log.info("unsupported datatype") + null + } + } + } + } + + override def newInstance() = OrcRelation(path, conf, sqlContext).asInstanceOf[this.type] +} + +private[sql] object OrcRelation { + + + // The orc compression short names + val shortOrcCompressionCodecNames = Map( + "NONE" -> CompressionKind.NONE, + "UNCOMPRESSED" -> CompressionKind.NONE, + "SNAPPY" -> CompressionKind.SNAPPY, + "ZLIB" -> CompressionKind.ZLIB, + "LZO" -> CompressionKind.LZO) + + /** + * Creates a new OrcRelation and underlying Orcfile for the given LogicalPlan. Note that + * this is used inside [[org.apache.spark.sql.execution.SparkStrategies]] to + * create a resolved relation as a data sink for writing to a Orcfile. + * + * @param pathString The directory the ORCfile will be stored in. + * @param child The child node that will be used for extracting the schema. + * @param conf A configuration to be used. + * @return An empty OrcRelation with inferred metadata. + */ + def create(pathString: String, + child: LogicalPlan, + conf: Configuration, + sqlContext: SQLContext): OrcRelation = { + if (!child.resolved) { + throw new UnresolvedException[LogicalPlan]( + child, + "Attempt to create Orc table from unresolved child") + } + createEmpty(pathString, child.output, false, conf, sqlContext) + } + + /** + * Creates an empty OrcRelation and underlying Orcfile that only + * consists of the Metadata for the given schema. + * + * @param pathString The directory the Orcfile will be stored in. + * @param attributes The schema of the relation. + * @param conf A configuration to be used. + * @return An empty OrcRelation. + */ + def createEmpty(pathString: String, + attributes: Seq[Attribute], + allowExisting: Boolean, + conf: Configuration, + sqlContext: SQLContext): OrcRelation = { + val path = checkPath(pathString, allowExisting, conf) + + /** set compression kind in hive 0.13.1 + * conf.set( + * HiveConf.ConfVars.OHIVE_ORC_DEFAULT_COMPRESS.varname, + * shortOrcCompressionCodecNames.getOrElse( + * sqlContext.orcCompressionCodec.toUpperCase, CompressionKind.NONE).name) + */ + val orcRelation = new OrcRelation(path.toString, Some(conf), sqlContext) + + orcRelation + } + + private def checkPath(pathStr: String, allowExisting: Boolean, conf: Configuration): Path = { + if (pathStr == null) { + throw new IllegalArgumentException("Unable to create OrcRelation: path is null") + } + val origPath = new Path(pathStr) + val fs = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException( + s"Unable to create OrcRelation: incorrectly formatted path $pathStr") + } + val path = origPath.makeQualified(fs) + if (!allowExisting && fs.exists(path)) { + sys.error(s"File $pathStr already exists.") + } + + if (fs.exists(path) && + !fs.getFileStatus(path) + .getPermission + .getUserAction + .implies(FsAction.READ_WRITE)) { + throw new IOException( + s"Unable to create OrcRelation: path $path not read-writable") + } + path + } +} + +private[sql] object OrcFileOperator { + final val COMPRESSION: String = "orcfiles.compression" + + /** + * + * @param origPath + * @return + */ + def readMetaData(origPath: Path): Reader = { + val job = new Job() + val conf = ContextUtil.getConfiguration(job).asInstanceOf[JobConf] + val fs: FileSystem = origPath.getFileSystem(conf) + val orcFiles = FileSystemHelper.listFiles(origPath, conf, ".orc") + if (orcFiles != Seq.empty) { +// NewFileInputFormat.setInputPaths(conf, orcFiles(0))// why set inputpath here + if (fs.exists(origPath)) { + OrcFile.createReader(fs, orcFiles(0)) + } else { + null + } + } else { + null + } + } + + + def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) { + if (origPath == null) { + throw new IllegalArgumentException("Unable to write Parquet metadata: path is null") + } + val fs = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException( + s"Unable to write Orc metadata: path $origPath is incorrectly formatted") + } + + val path = origPath.makeQualified(fs) + if (fs.exists(path) && !fs.getFileStatus(path).isDir) { + throw new IllegalArgumentException(s"Expected to write to directory $path but found file") + } + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcTableOperations.scala new file mode 100644 index 000000000000..bb1f342cc9ee --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcTableOperations.scala @@ -0,0 +1,418 @@ +/* + * 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.orc + +import org.apache.spark.sql.execution.{ExistingRdd, LeafNode, UnaryNode, SparkPlan} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.{TaskContext, SerializableWritable} +import org.apache.spark.rdd.RDD + +import _root_.parquet.hadoop.util.ContextUtil +import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, FileOutputCommitter} +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat +import org.apache.hadoop.io.{Writable, NullWritable} +import org.apache.hadoop.mapreduce.{TaskID, TaskAttemptContext, Job} + +import org.apache.hadoop.hive.ql.io.orc.{OrcFile, OrcSerde, OrcInputFormat, OrcOutputFormat} +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils +import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} + +import java.io.IOException +import java.text.SimpleDateFormat +import java.util.{Locale, Date} +import scala.collection.JavaConversions._ +import org.apache.hadoop.mapred.{SparkHadoopMapRedUtil, Reporter, JobConf} + +/** + * orc table scan operator. Imports the file that backs the given + * [[org.apache.spark.sql.orc.OrcRelation]] as a ``RDD[Row]``. + */ +case class OrcTableScan( + output: Seq[Attribute], + relation: OrcRelation, + columnPruningPred: Option[Expression]) + extends LeafNode { + + @transient + lazy val serde: OrcSerde = initSerde + + @transient + lazy val getFieldValue: Seq[Product => Any] = { + val inspector = serde.getObjectInspector.asInstanceOf[StructObjectInspector] + output.map(attr => { + val ref = inspector.getStructFieldRef(attr.name.toLowerCase(Locale.ENGLISH)) + row: Product => { + val fieldData = row.productElement(1) + val data = inspector.getStructFieldData(fieldData, ref) + unwrapData(data, ref.getFieldObjectInspector) + } + }) + } + + private def initSerde(): OrcSerde = { + val serde = new OrcSerde + serde.initialize(null, relation.prop) + serde + } + + def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { + case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) + case li: ListObjectInspector => + Option(li.getList(data)) + .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) + .orNull + case mi: MapObjectInspector => + Option(mi.getMap(data)).map( + _.map { + case (k, v) => + (unwrapData(k, mi.getMapKeyObjectInspector), + unwrapData(v, mi.getMapValueObjectInspector)) + }.toMap).orNull + case si: StructObjectInspector => + val allRefs = si.getAllStructFieldRefs + new GenericRow( + allRefs.map(r => + unwrapData(si.getStructFieldData(data, r), r.getFieldObjectInspector)).toArray) + } + + override def execute(): RDD[Row] = { + val sc = sqlContext.sparkContext + val job = new Job(sc.hadoopConfiguration) + + val conf: Configuration = ContextUtil.getConfiguration(job) + val fileList = FileSystemHelper.listFiles(relation.path, conf) + + // add all paths in the directory but skip "hidden" ones such + // as "_SUCCESS" + for (path <- fileList if !path.getName.startsWith("_")) { + FileInputFormat.addInputPath(job, path) + } + val serialConf = sc.broadcast(new SerializableWritable(conf)) + + setColumnIds(output, relation, conf) + val inputClass = classOf[OrcInputFormat].asInstanceOf[ + Class[_ <: org.apache.hadoop.mapred.InputFormat[Void, Row]]] + + val rowRdd = productToRowRdd(sc.hadoopRDD[Void, Row]( + serialConf.value.value.asInstanceOf[JobConf], inputClass, classOf[Void], classOf[Row])) + rowRdd + } + + /** + * @param output + * @param relation + * @param conf + */ + def setColumnIds(output: Seq[Attribute], relation: OrcRelation, conf: Configuration) { + val idBuff = new StringBuilder() + + output.map(att => { + val realName = att.name.toLowerCase(Locale.ENGLISH) + val id = relation.fieldIdCache.getOrElse(realName, null) + if (null != id) { + idBuff.append(id) + idBuff.append(",") + } + }) + if (idBuff.length > 0) { + idBuff.setLength(idBuff.length - 1) + } + conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, idBuff.toString()) + } + + /** + * + * @param data + * @tparam A + * @return + */ + def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { + data.mapPartitions { + iterator => + if (iterator.isEmpty) { + Iterator.empty + } else { + val bufferedIterator = iterator.buffered + bufferedIterator.map {r => + val values = getFieldValue.map(_(r)) + new GenericRow(values.map { + case n: String if n.toLowerCase == "null" => "" + case varchar: HiveVarchar => varchar.getValue + case decimal: HiveDecimal => + BigDecimal(decimal.bigDecimalValue) + case null => "" + case other => other + + }.toArray) + } + } + } + } + + /** + * Applies a (candidate) projection. + * + * @param prunedAttributes The list of attributes to be used in the projection. + * @return Pruned TableScan. + */ + def pruneColumns(prunedAttributes: Seq[Attribute]): OrcTableScan = { + OrcTableScan(prunedAttributes, relation, columnPruningPred) + } +} + +/** + * + * @param relation + * @param child + * @param overwrite + */ +private[sql] case class InsertIntoOrcTable( + relation: OrcRelation, + child: SparkPlan, + overwrite: Boolean = false) + extends UnaryNode with SparkHadoopMapRedUtil with org.apache.spark.Logging { + + override def output = child.output + + val intputClass: Class[_] = getInputClass + + @transient val sc = sqlContext.sparkContext + + val inspector = sc.broadcast(getInspector(intputClass)) + + @transient lazy val orcSerde = initFieldInfo + + private def getInputClass: Class[_] = { + val existRdd = child.asInstanceOf[ExistingRdd] + val productClass = existRdd.rdd.firstParent.elementClassTag.runtimeClass + logInfo("productClass is " + productClass) + val clazz = productClass + if (null == relation.rowClass) { + relation.rowClass = clazz + } + clazz + } + + private def getInspector(clazz: Class[_]): ObjectInspector = { + val inspector = ObjectInspectorFactory.getReflectionObjectInspector(clazz, + ObjectInspectorFactory.ObjectInspectorOptions.JAVA) + inspector + } + + private def initFieldInfo(): OrcSerde = { + val serde: OrcSerde = new OrcSerde + serde + } + + /** + * Inserts all rows into the Orc file. + */ + override def execute() = { + val childRdd = child.execute() + assert(childRdd != null) + + val job = new Job(sqlContext.sparkContext.hadoopConfiguration) + // TODO: move that to function in object + val conf = job.getConfiguration + + val fspath = new Path(relation.path) + val fs = fspath.getFileSystem(conf) + + if (overwrite) { + try { + fs.delete(fspath, true) + } catch { + case e: IOException => + throw new IOException( + s"Unable to clear output directory ${fspath.toString} prior" + + s" to InsertIntoOrcTable:\n${e.toString}") + } + } + + val existRdd = child.asInstanceOf[ExistingRdd] + val parentRdd = existRdd.rdd.firstParent[Product] + val writableRdd = parentRdd.map(obj => { + orcSerde.serialize(obj, inspector.value) + }) + + saveAsHadoopFile(writableRdd, relation.rowClass, relation.path, conf) + + // We return the child RDD to allow chaining (alternatively, one could return nothing). + childRdd + } + + + // based on ``saveAsNewAPIHadoopFile`` in [[PairRDDFunctions]] + // TODO: Maybe PairRDDFunctions should use Product2 instead of Tuple2? + // .. then we could use the default one and could use [[MutablePair]] + // instead of ``Tuple2`` + private def saveAsHadoopFile( + rdd: RDD[Writable], + rowClass: Class[_], + path: String, + @transient conf: Configuration) { + val job = new Job(conf) + val keyType = classOf[Void] + job.setOutputKeyClass(keyType) + job.setOutputValueClass(classOf[Writable]) + FileOutputFormat.setOutputPath(job, new Path(path)) + + val wrappedConf = new SerializableWritable(job.getConfiguration) + + val formatter = new SimpleDateFormat("yyyyMMddHHmm") + val jobtrackerID = formatter.format(new Date()) + val stageId = sqlContext.sparkContext.newRddId() + + val taskIdOffset = + if (overwrite) { + 1 + } else { + FileSystemHelper + .findMaxTaskId(FileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1 + } + + def getWriter( + outFormat: OrcOutputFormat, + conf: Configuration, + path: Path, + reporter: Reporter) = { + val fs = path.getFileSystem(conf) + outFormat.getRecordWriter(fs, conf.asInstanceOf[JobConf], path.toUri.getPath, reporter). + asInstanceOf[org.apache.hadoop.mapred.RecordWriter[NullWritable, Writable]] + } + + def getCommitterAndWriter(offset: Int, context: TaskAttemptContext) = { + val outFormat = new OrcOutputFormat + + val taskId: TaskID = context.getTaskAttemptID.getTaskID + val partition: Int = taskId.getId + val filename = s"part-r-${partition + offset}.orc" + val output: Path = FileOutputFormat.getOutputPath(context) + val committer = new FileOutputCommitter(output, context) + val path = new Path(committer.getWorkPath, filename) + val writer = getWriter(outFormat, wrappedConf.value, path, Reporter.NULL) + (committer, writer) + } + + def writeShard(context: TaskContext, iter: Iterator[Writable]): Int = { + // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it + // around by taking a mod. We expect that no task will be attempted 2 billion times. + val attemptNumber = (context.attemptId % Int.MaxValue).toInt + /* "reduce task" */ + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, + attemptNumber) + val hadoopContext = newTaskAttemptContext(wrappedConf.value.asInstanceOf[JobConf], attemptId) + val workerAndComitter = getCommitterAndWriter(taskIdOffset, hadoopContext) + val writer = workerAndComitter._2 + + while (iter.hasNext) { + val row = iter.next() + writer.write(NullWritable.get(), row) + } + + writer.close(Reporter.NULL) + workerAndComitter._1.commitTask(hadoopContext) + return 1 + } + + /* apparently we need a TaskAttemptID to construct an OutputCommitter; + * however we're only going to use this local OutputCommitter for + * setupJob/commitJob, so we just use a dummy "map" task. + */ + val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0) + val jobTaskContext = newTaskAttemptContext( + wrappedConf.value.asInstanceOf[JobConf], jobAttemptId) + val workerAndComitter = getCommitterAndWriter(taskIdOffset, jobTaskContext) + workerAndComitter._1.setupJob(jobTaskContext) + sc.runJob(rdd, writeShard _) + workerAndComitter._1.commitJob(jobTaskContext) + } + +} + +/** + * + */ +private[orc] object FileSystemHelper { + /** + * + * @param pathStr + * @param conf + * @return + */ + def listFiles(pathStr: String, conf: Configuration): Seq[Path] = { + val origPath = new Path(pathStr) + val fs = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException( + s"OrcTableOperations: Path $origPath is incorrectly formatted") + } + val path = origPath.makeQualified(fs) + if (!fs.exists(path) || !fs.getFileStatus(path).isDir) { + Seq.empty + } else { + fs.listStatus(path).map(_.getPath) + } + } + + /** + * + * @param origPath + * @param conf + * @param extension + * @return + */ + def listFiles(origPath: Path, conf: Configuration, extension: String): Seq[Path] = { + val fs = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException( + s"OrcTableOperations: Path $origPath is incorrectly formatted") + } + val path = origPath.makeQualified(fs) + if (fs.exists(path) && fs.getFileStatus(path).isDir) { + fs.listStatus(path).map(_.getPath).filter(p => p.getName.endsWith(extension)) + } else { + Seq.empty + } + } + + /** + * Finds the maximum taskid in the output file names at the given path. + */ + def findMaxTaskId(pathStr: String, conf: Configuration): Int = { + val files = FileSystemHelper.listFiles(pathStr, conf) + // filename pattern is part-r-.orc + val nameP = new scala.util.matching.Regex( """part-r-(\d{1,}).orc""", "taskid") + val hiddenFileP = new scala.util.matching.Regex("_.*") + files.map(_.getName).map { + case nameP(taskid) => taskid.toInt + case hiddenFileP() => 0 + case other: String => { + sys.error("ERROR: attempting to append to set of Orc files and found file" + + s"that does not match name pattern: $other") + 0 + } + case _ => 0 + }.reduceLeft((a, b) => if (a < b) b else a) + } +} + From ec3cdaf6fbee283107e409356d115757fbfc1ff4 Mon Sep 17 00:00:00 2001 From: scwf Date: Mon, 29 Sep 2014 21:45:55 +0800 Subject: [PATCH 02/18] add unit tests --- .../apache/spark/sql/orc/ORCQuerySuite.scala | 184 ++++++++++++++++++ 1 file changed, 184 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/orc/ORCQuerySuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/orc/ORCQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/orc/ORCQuerySuite.scala new file mode 100644 index 000000000000..6daf3b5855f4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/orc/ORCQuerySuite.scala @@ -0,0 +1,184 @@ +/* + * 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.orc + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.io.orc.CompressionKind +import org.apache.spark.sql.{SQLConf, SchemaRDD, TestData, QueryTest} +import org.apache.spark.sql.test.TestSQLContext +import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} +import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.test.TestSQLContext._ + +import java.io.File + +case class TestRDDEntry(key: Int, value: String) + +case class NullReflectData( + intField: java.lang.Integer, + longField: java.lang.Long, + floatField: java.lang.Float, + doubleField: java.lang.Double, + booleanField: java.lang.Boolean) + +case class OptionalReflectData( + intField: Option[Int], + longField: Option[Long], + floatField: Option[Float], + doubleField: Option[Double], + booleanField: Option[Boolean]) + +case class Nested(i: Int, s: String) + +case class Data(array: Seq[Int], nested: Nested) + +case class AllDataTypes( + stringField: String, + intField: Int, + longField: Long, + floatField: Float, + doubleField: Double, + shortField: Short, + byteField: Byte, + booleanField: Boolean) + +case class AllDataTypesWithNonPrimitiveType( + stringField: String, + intField: Int, + longField: Long, + floatField: Float, + doubleField: Double, + shortField: Short, + byteField: Byte, + booleanField: Boolean, + array: Seq[Int], + arrayContainsNull: Seq[Option[Int]], + map: Map[Int, Long], + mapValueContainsNull: Map[Int, Option[Long]], + data: Data) + +case class BinaryData(binaryData: Array[Byte]) + +class OrcQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { + TestData // Load test data tables. + + var testRDD: SchemaRDD = null + test("Read/Write All Types") { + val tempDir = getTempFilePath("orcTest").getCanonicalPath + val range = (0 to 255) + val data = sparkContext.parallelize(range) + .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + + data.saveAsOrcFile(tempDir) + + checkAnswer( + orcFile(tempDir), + data.toSchemaRDD.collect().toSeq) + + Utils.deleteRecursively(new File(tempDir)) + + } + + test("Compression options for writing to a Orcfile") { + val defaultOrcCompressionCodec = TestSQLContext.orcCompressionCodec + //TODO: support other compress codec + val file = getTempFilePath("orcTest") + val path = file.toString + val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + + // test default compression codec, now only support zlib + rdd.saveAsOrcFile(path) + var actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name + assert(actualCodec == TestSQLContext.orcCompressionCodec.toUpperCase) + + /** + orcFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "UNCOMPRESSED" + TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, "UNCOMPRESSED") + + rdd.saveAsOrcFile(path) + actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name + assert(actualCodec == OrcRelation.shortOrcCompressionCodecNames.getOrElse( + TestSQLContext.orcCompressionCodec.toUpperCase, CompressionKind.NONE).name.toUpperCase) + + orcFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "none" + TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, "none") + + rdd.saveAsOrcFile(path) + actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name() + assert(actualCodec ==TestSQLContext.orcCompressionCodec.toUpperCase) + + orcFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test gzip compression codec + TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, "zlib") + + rdd.saveAsOrcFile(path) + actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name() + assert(actualCodec ==TestSQLContext.orcCompressionCodec.toUpperCase) + + orcFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test snappy compression codec + TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, "snappy") + + rdd.saveAsOrcFile(path) + actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name() + assert(actualCodec ==TestSQLContext.orcCompressionCodec.toUpperCase) + + orcFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, defaultOrcCompressionCodec) + */ + } +} From 71262909ac01f178c55d8db113b2c6a12fdd1654 Mon Sep 17 00:00:00 2001 From: scwf Date: Mon, 29 Sep 2014 22:25:42 +0800 Subject: [PATCH 03/18] add orc to example of spark sql --- .../org/apache/spark/examples/sql/RDDRelation.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index d56d64c56420..7610d36c698e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -59,14 +59,25 @@ object RDDRelation { // Write out an RDD as a parquet file. rdd.saveAsParquetFile("pair.parquet") + // Write out an RDD as a orc file. + rdd.saveAsOrcFile("pair.orc") + + // Read in parquet file. Parquet files are self-describing so the schmema is preserved. val parquetFile = sqlContext.parquetFile("pair.parquet") + // Read in orc file. orc files are self-describing so the schmema is preserved. + val orcFile = sqlContext.orcFile("pair.orc") + + // Queries can be run using the DSL on parequet files just like the original RDD. parquetFile.where('key === 1).select('value as 'a).collect().foreach(println) // These files can also be registered as tables. parquetFile.registerTempTable("parquetFile") sql("SELECT * FROM parquetFile").collect().foreach(println) + + orcFile.registerTempTable("orcFile") + sql("SELECT * FROM orcFile").collect().foreach(println) } } From 1505af48becdca9f17d84d9c7a0ef7f03dbc4e8a Mon Sep 17 00:00:00 2001 From: scwf Date: Sat, 4 Oct 2014 15:34:27 +0800 Subject: [PATCH 04/18] fix according comments and move orc to hive sub project --- .../spark/examples/sql/RDDRelation.scala | 11 -- .../examples/sql/hive/HiveFromSpark.scala | 10 + sql/core/pom.xml | 15 -- .../scala/org/apache/spark/sql/SQLConf.scala | 7 - .../org/apache/spark/sql/SQLContext.scala | 46 +---- .../org/apache/spark/sql/SchemaRDDLike.scala | 4 +- .../spark/sql/execution/SparkStrategies.scala | 22 --- .../sql/parquet/ParquetTableOperations.scala | 40 +++- .../apache/spark/sql/orc/ORCQuerySuite.scala | 184 ------------------ .../apache/spark/sql/hive/HiveContext.scala | 50 ++++- .../spark/sql/hive/HiveStrategies.scala | 21 ++ .../spark/sql/hive}/orc/OrcRelation.scala | 93 ++++----- .../sql/hive}/orc/OrcTableOperations.scala | 166 +++++----------- .../spark/sql/hive/orc/ORCQuerySuite.scala | 74 +++++++ 14 files changed, 274 insertions(+), 469 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/orc/ORCQuerySuite.scala rename sql/{core/src/main/scala/org/apache/spark/sql => hive/src/main/scala/org/apache/spark/sql/hive}/orc/OrcRelation.scala (79%) rename sql/{core/src/main/scala/org/apache/spark/sql => hive/src/main/scala/org/apache/spark/sql/hive}/orc/OrcTableOperations.scala (71%) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index c882d40c424e..2e98b2dc30b8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -59,17 +59,9 @@ object RDDRelation { // Write out an RDD as a parquet file. rdd.saveAsParquetFile("pair.parquet") - // Write out an RDD as a orc file. - rdd.saveAsOrcFile("pair.orc") - - // Read in parquet file. Parquet files are self-describing so the schmema is preserved. val parquetFile = sqlContext.parquetFile("pair.parquet") - // Read in orc file. orc files are self-describing so the schmema is preserved. - val orcFile = sqlContext.orcFile("pair.orc") - - // Queries can be run using the DSL on parequet files just like the original RDD. parquetFile.where('key === 1).select('value as 'a).collect().foreach(println) @@ -77,9 +69,6 @@ object RDDRelation { parquetFile.registerTempTable("parquetFile") sql("SELECT * FROM parquetFile").collect().foreach(println) - orcFile.registerTempTable("orcFile") - sql("SELECT * FROM orcFile").collect().foreach(println) - sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index e26f213e8afa..ea958eddb8f8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -62,6 +62,16 @@ object HiveFromSpark { println("Result of SELECT *:") sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) + // Write out an RDD as a orc file. + rdd.saveAsOrcFile("pair.orc") + + // Read in orc file. Orc files are self-describing so the schmema is preserved. + val orcFile = hiveContext.orcFile("pair.orc") + + // These files can also be registered as tables. + orcFile.registerTempTable("orcFile") + sql("SELECT * FROM records r JOIN orcFile s ON r.key = s.key").collect().foreach(println) + sc.stop() } } diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 724d632c480b..bd110218d34f 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -53,21 +53,6 @@ test-jar test - - org.spark-project.hive - hive-exec - ${hive.version} - - - commons-logging - commons-logging - - - org.slf4j - slf4j-api - - - com.twitter parquet-column diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 1fca2f380e7f..f6f4cf3b80d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -35,7 +35,6 @@ private[spark] object SQLConf { val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec" - val ORC_COMPRESSION = "spark.sql.orc.compression.codec" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -84,12 +83,6 @@ private[sql] trait SQLConf { /** The compression codec for writing to a Parquetfile */ private[spark] def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION, "snappy") - /** The compression codec for writing to a Orcfile - * Note: only support zlib now since we use ```OrcOutputFormat.getRecordWriter``` ,which is not - * allowed to configure thr compression kind - */ - private[spark] def orcCompressionCodec: String = getConf(ORC_COMPRESSION, "zlib") - /** The number of rows that will be */ private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "1000").toInt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7bf92b92d6b9..10331f8c7c85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -36,8 +36,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.SparkStrategies import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.sql.orc.OrcRelation +import org.apache.spark.SparkContext /** * :: AlphaComponent :: @@ -148,14 +147,6 @@ class SQLContext(@transient val sparkContext: SparkContext) def parquetFile(path: String): SchemaRDD = new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) - /** - * Loads a Orc file, returning the result as a [[SchemaRDD]]. - * - * @group userf - */ - def orcFile(path: String): SchemaRDD = - new SchemaRDD(this, orc.OrcRelation(path, Some(sparkContext.hadoopConfiguration), this)) - /** * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. * It goes through the entire dataset once to determine the schema. @@ -255,40 +246,6 @@ class SQLContext(@transient val sparkContext: SparkContext) path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) } - /** - * :: Experimental :: - * Creates an empty parquet file with the schema of class `A`, which can be registered as a table. - * This registered table can be used as the target of future `insertInto` operations. - * - * {{{ - * val sqlContext = new SQLContext(...) - * import sqlContext._ - * - * case class Person(name: String, age: Int) - * createOrcFile[Person]("path/to/file.orc").registerTempTable("people") - * sql("INSERT INTO people SELECT 'michael', 29") - * }}} - * - * @tparam A A case class type that describes the desired schema of the parquet file to be - * created. - * @param path The path where the directory containing parquet metadata should be created. - * Data inserted into this table will also be stored at this location. - * @param allowExisting When false, an exception will be thrown if this directory already exists. - * @param conf A Hadoop configuration object that can be used to specify options to the parquet - * output format. - * - * @group userf - */ - @Experimental - def createOrcFile[A <: Product : TypeTag]( - path: String, - allowExisting: Boolean = true, - conf: Configuration = new Configuration()): SchemaRDD = { - new SchemaRDD( - this, - OrcRelation.createEmpty(path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) - } - /** * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only * during the lifetime of this instance of SQLContext. @@ -334,7 +291,6 @@ class SQLContext(@transient val sparkContext: SparkContext) HashJoin :: InMemoryScans :: ParquetOperations :: - OrcOperations:: BasicOperators :: CartesianProduct :: BroadcastNestedLoopJoin :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index a42ec3d3ea35..248b7c0d5af2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -18,9 +18,8 @@ package org.apache.spark.sql import org.apache.spark.annotation.{DeveloperApi, Experimental} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.LogicalRDD /** @@ -81,6 +80,7 @@ private[sql] trait SchemaRDDLike { * Saves the contents of this `SchemaRDD` as a orc file, preserving the schema. Files that * are written out using this method can be read back in as a SchemaRDD using the `orcFile` * function. + * Note: you can only use it in HiveContext * * @group schema */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index d04c908c16d2..cf93d5ad7b50 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.parquet._ -import org.apache.spark.sql.orc.{OrcTableScan, InsertIntoOrcTable, OrcRelation} private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => @@ -238,27 +237,6 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } - object OrcOperations extends Strategy { - def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.WriteToOrcFile(path, child) => - val relation = - OrcRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) - InsertIntoOrcTable(relation, planLater(child), overwrite=true) :: Nil - case logical.InsertIntoOrcTable(table: OrcRelation, partition, child, overwrite) => - InsertIntoOrcTable(table, planLater(child), overwrite) :: Nil - case PhysicalOperation(projectList, filters, relation: OrcRelation) => - // TODO: need to implement predict push down. - val prunePushedDownFilters = identity[Seq[Expression]] _ - pruneFilterProject( - projectList, - filters, - prunePushedDownFilters, - OrcTableScan(_, relation, None)) :: Nil - - case _ => Nil - } - } - object InMemoryScans extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(projectList, filters, mem: InMemoryRelation) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index ffb732347d30..eed212cc0819 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -283,7 +283,7 @@ case class InsertIntoParquetTable( 1 } else { FileSystemHelper - .findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1 + .findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration, "parquet") + 1 } def writeShard(context: TaskContext, iter: Iterator[Row]): Int = { @@ -488,7 +488,7 @@ private[parquet] object FilteringParquetRowInputFormat { .build[FileStatus, Array[BlockLocation]]() } -private[parquet] object FileSystemHelper { +private[sql] object FileSystemHelper { def listFiles(pathStr: String, conf: Configuration): Seq[Path] = { val origPath = new Path(pathStr) val fs = origPath.getFileSystem(conf) @@ -504,19 +504,41 @@ private[parquet] object FileSystemHelper { fs.listStatus(path).map(_.getPath) } - /** - * Finds the maximum taskid in the output file names at the given path. - */ - def findMaxTaskId(pathStr: String, conf: Configuration): Int = { + /** + * List files with special extension + */ + def listFiles(origPath: Path, conf: Configuration, extension: String): Seq[Path] = { + val fs = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException( + s"OrcTableOperations: Path $origPath is incorrectly formatted") + } + val path = origPath.makeQualified(fs) + if (fs.exists(path) && fs.getFileStatus(path).isDir) { + fs.listStatus(path).map(_.getPath).filter(p => p.getName.endsWith(extension)) + } else { + Seq.empty + } + } + + /** + * Finds the maximum taskid in the output file names at the given path. + */ + def findMaxTaskId(pathStr: String, conf: Configuration, extension: String): Int = { val files = FileSystemHelper.listFiles(pathStr, conf) - // filename pattern is part-r-.parquet - val nameP = new scala.util.matching.Regex("""part-r-(\d{1,}).parquet""", "taskid") + // filename pattern is part-r-.$extension + val nameP = extension match { + case "parquet" => new scala.util.matching.Regex( """part-r-(\d{1,}).parquet""", "taskid") + case "orc" => new scala.util.matching.Regex( """part-r-(\d{1,}).orc""", "taskid") + case _ => + sys.error(s"ERROR: unsupported extension: $extension") + } val hiddenFileP = new scala.util.matching.Regex("_.*") files.map(_.getName).map { case nameP(taskid) => taskid.toInt case hiddenFileP() => 0 case other: String => { - sys.error("ERROR: attempting to append to set of Parquet files and found file" + + sys.error(s"ERROR: attempting to append to set of $extension files and found file" + s"that does not match name pattern: $other") 0 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/orc/ORCQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/orc/ORCQuerySuite.scala deleted file mode 100644 index 6daf3b5855f4..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/orc/ORCQuerySuite.scala +++ /dev/null @@ -1,184 +0,0 @@ -/* - * 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.orc - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.ql.io.orc.CompressionKind -import org.apache.spark.sql.{SQLConf, SchemaRDD, TestData, QueryTest} -import org.apache.spark.sql.test.TestSQLContext -import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} -import org.apache.spark.util.Utils -import org.apache.spark.sql.catalyst.util.getTempFilePath -import org.apache.spark.sql.test.TestSQLContext._ - -import java.io.File - -case class TestRDDEntry(key: Int, value: String) - -case class NullReflectData( - intField: java.lang.Integer, - longField: java.lang.Long, - floatField: java.lang.Float, - doubleField: java.lang.Double, - booleanField: java.lang.Boolean) - -case class OptionalReflectData( - intField: Option[Int], - longField: Option[Long], - floatField: Option[Float], - doubleField: Option[Double], - booleanField: Option[Boolean]) - -case class Nested(i: Int, s: String) - -case class Data(array: Seq[Int], nested: Nested) - -case class AllDataTypes( - stringField: String, - intField: Int, - longField: Long, - floatField: Float, - doubleField: Double, - shortField: Short, - byteField: Byte, - booleanField: Boolean) - -case class AllDataTypesWithNonPrimitiveType( - stringField: String, - intField: Int, - longField: Long, - floatField: Float, - doubleField: Double, - shortField: Short, - byteField: Byte, - booleanField: Boolean, - array: Seq[Int], - arrayContainsNull: Seq[Option[Int]], - map: Map[Int, Long], - mapValueContainsNull: Map[Int, Option[Long]], - data: Data) - -case class BinaryData(binaryData: Array[Byte]) - -class OrcQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { - TestData // Load test data tables. - - var testRDD: SchemaRDD = null - test("Read/Write All Types") { - val tempDir = getTempFilePath("orcTest").getCanonicalPath - val range = (0 to 255) - val data = sparkContext.parallelize(range) - .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) - - data.saveAsOrcFile(tempDir) - - checkAnswer( - orcFile(tempDir), - data.toSchemaRDD.collect().toSeq) - - Utils.deleteRecursively(new File(tempDir)) - - } - - test("Compression options for writing to a Orcfile") { - val defaultOrcCompressionCodec = TestSQLContext.orcCompressionCodec - //TODO: support other compress codec - val file = getTempFilePath("orcTest") - val path = file.toString - val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) - .map(i => TestRDDEntry(i, s"val_$i")) - - // test default compression codec, now only support zlib - rdd.saveAsOrcFile(path) - var actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name - assert(actualCodec == TestSQLContext.orcCompressionCodec.toUpperCase) - - /** - orcFile(path).registerTempTable("tmp") - checkAnswer( - sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), - (5, "val_5") :: - (7, "val_7") :: Nil) - - Utils.deleteRecursively(file) - - // test uncompressed parquet file with property value "UNCOMPRESSED" - TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, "UNCOMPRESSED") - - rdd.saveAsOrcFile(path) - actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name - assert(actualCodec == OrcRelation.shortOrcCompressionCodecNames.getOrElse( - TestSQLContext.orcCompressionCodec.toUpperCase, CompressionKind.NONE).name.toUpperCase) - - orcFile(path).registerTempTable("tmp") - checkAnswer( - sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), - (5, "val_5") :: - (7, "val_7") :: Nil) - - Utils.deleteRecursively(file) - - // test uncompressed parquet file with property value "none" - TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, "none") - - rdd.saveAsOrcFile(path) - actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name() - assert(actualCodec ==TestSQLContext.orcCompressionCodec.toUpperCase) - - orcFile(path).registerTempTable("tmp") - checkAnswer( - sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), - (5, "val_5") :: - (7, "val_7") :: Nil) - - Utils.deleteRecursively(file) - - // test gzip compression codec - TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, "zlib") - - rdd.saveAsOrcFile(path) - actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name() - assert(actualCodec ==TestSQLContext.orcCompressionCodec.toUpperCase) - - orcFile(path).registerTempTable("tmp") - checkAnswer( - sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), - (5, "val_5") :: - (7, "val_7") :: Nil) - - Utils.deleteRecursively(file) - - // test snappy compression codec - TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, "snappy") - - rdd.saveAsOrcFile(path) - actualCodec = OrcFileOperator.readMetaData(new Path(path)).getCompression.name() - assert(actualCodec ==TestSQLContext.orcCompressionCodec.toUpperCase) - - orcFile(path).registerTempTable("tmp") - checkAnswer( - sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), - (5, "val_5") :: - (7, "val_7") :: Nil) - - Utils.deleteRecursively(file) - - TestSQLContext.setConf(SQLConf.ORC_COMPRESSION, defaultOrcCompressionCodec) - */ - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 8bcc098bbb62..b959070e4764 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConversions._ import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf @@ -37,14 +38,14 @@ import org.apache.hadoop.hive.serde2.io.TimestampWritable import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.hive.orc.OrcRelation import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators} import org.apache.spark.sql.catalyst.analysis.{OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.ExtractPythonUdfs -import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.execution.{Command => PhysicalCommand} +import org.apache.spark.sql.execution.{Command => PhysicalCommand, ExtractPythonUdfs, QueryExecutionException} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand /** @@ -120,6 +121,48 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.createTable("default", tableName, ScalaReflection.attributesFor[A], allowExisting) } + /** + * Loads a Orc file, returning the result as a [[SchemaRDD]]. + * + * @group userf + */ + def orcFile(path: String): SchemaRDD = + new SchemaRDD(this, orc.OrcRelation(path, Some(sparkContext.hadoopConfiguration), this)) + + /** + * :: Experimental :: + * Creates an empty orc file with the schema of class `A`, which can be registered as a table. + * This registered table can be used as the target of future `insertInto` operations. + * + * {{{ + * val sqlContext = new SQLContext(...) + * import sqlContext._ + * + * case class Person(name: String, age: Int) + * createOrcFile[Person]("path/to/file.orc").registerTempTable("people") + * sql("INSERT INTO people SELECT 'michael', 29") + * }}} + * + * @tparam A A case class type that describes the desired schema of the orc file to be + * created. + * @param path The path where the directory containing parquet metadata should be created. + * Data inserted into this table will also be stored at this location. + * @param allowExisting When false, an exception will be thrown if this directory already exists. + * @param conf A Hadoop configuration object that can be used to specify options to the parquet + * output format. + * + * @group userf + */ + @Experimental + def createOrcFile[A <: Product : TypeTag]( + path: String, + allowExisting: Boolean = true, + conf: Configuration = new Configuration()): SchemaRDD = { + new SchemaRDD( + this, + OrcRelation.createEmpty(path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) + } + /** * Analyzes the given table in the current database to generate statistics, which will be * used in query optimizations. @@ -333,6 +376,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { HiveCommandStrategy(self), TakeOrdered, ParquetOperations, + OrcOperations, InMemoryScans, ParquetConversion, // Must be before HiveTableScans HiveTableScans, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 508d8239c762..14f63d2ecb0a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ +import org.apache.spark.sql.hive.orc.{InsertIntoOrcTable, OrcTableScan, OrcRelation} import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.{SQLContext, SchemaRDD} @@ -221,4 +222,24 @@ private[hive] trait HiveStrategies { case _ => Nil } } + + object OrcOperations extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case logical.WriteToOrcFile(path, child) => + val relation = + OrcRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) + InsertIntoOrcTable(relation, planLater(child), overwrite=true) :: Nil + case logical.InsertIntoOrcTable(table: OrcRelation, partition, child, overwrite) => + InsertIntoOrcTable(table, planLater(child), overwrite) :: Nil + case PhysicalOperation(projectList, filters, relation: OrcRelation) => + // TODO: need to implement predict push down. + val prunePushedDownFilters = identity[Seq[Expression]] _ + pruneFilterProject( + projectList, + filters, + prunePushedDownFilters, + OrcTableScan(_, relation, None)) :: Nil + case _ => Nil + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala similarity index 79% rename from sql/core/src/main/scala/org/apache/spark/sql/orc/OrcRelation.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 8b5156c0f376..5b3460553aa7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -15,27 +15,28 @@ * limitations under the License. */ -package org.apache.spark.sql.orc +package org.apache.spark.sql.hive.orc + +import java.util.Properties +import java.io.IOException +import org.apache.hadoop.hive.ql.stats.StatsSetupConst + +import scala.collection.mutable -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedException, MultiInstanceRelation} -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.catalyst.types._ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.ql.io.orc._ -import org.apache.hadoop.mapred.{FileInputFormat => NewFileInputFormat, JobConf} import org.apache.hadoop.hive.ql.io.orc.OrcProto.Type.Kind -import org.apache.hadoop.mapreduce.Job -import parquet.hadoop.util.ContextUtil -import java.util.Properties -import java.io.IOException -import scala.collection.mutable + +import org.apache.spark.sql.parquet.FileSystemHelper import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedException, MultiInstanceRelation} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.types._ private[sql] case class OrcRelation( @@ -56,9 +57,11 @@ private[sql] case class OrcRelation( override val output = orcSchema + override lazy val statistics = Statistics(sizeInBytes = sqlContext.defaultSizeInBytes) + def orcSchema: Seq[Attribute] = { val origPath = new Path(path) - val reader = OrcFileOperator.readMetaData(origPath) + val reader = OrcFileOperator.readMetaData(origPath, conf) if (null != reader) { val inspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] @@ -98,9 +101,9 @@ private[sql] case class OrcRelation( } def convertToAttributes( - reader: Reader, - keys: java.util.List[String], - types: java.util.List[Integer]): Seq[Attribute] = { + reader: Reader, + keys: java.util.List[String], + types: java.util.List[Integer]): Seq[Attribute] = { val range = 0.until(keys.size()) range.map { i => reader.getTypes.get(types.get(i)).getKind match { @@ -132,16 +135,6 @@ private[sql] case class OrcRelation( } private[sql] object OrcRelation { - - - // The orc compression short names - val shortOrcCompressionCodecNames = Map( - "NONE" -> CompressionKind.NONE, - "UNCOMPRESSED" -> CompressionKind.NONE, - "SNAPPY" -> CompressionKind.SNAPPY, - "ZLIB" -> CompressionKind.ZLIB, - "LZO" -> CompressionKind.LZO) - /** * Creates a new OrcRelation and underlying Orcfile for the given LogicalPlan. Note that * this is used inside [[org.apache.spark.sql.execution.SparkStrategies]] to @@ -152,10 +145,11 @@ private[sql] object OrcRelation { * @param conf A configuration to be used. * @return An empty OrcRelation with inferred metadata. */ - def create(pathString: String, - child: LogicalPlan, - conf: Configuration, - sqlContext: SQLContext): OrcRelation = { + def create( + pathString: String, + child: LogicalPlan, + conf: Configuration, + sqlContext: SQLContext): OrcRelation = { if (!child.resolved) { throw new UnresolvedException[LogicalPlan]( child, @@ -173,19 +167,20 @@ private[sql] object OrcRelation { * @param conf A configuration to be used. * @return An empty OrcRelation. */ - def createEmpty(pathString: String, - attributes: Seq[Attribute], - allowExisting: Boolean, - conf: Configuration, - sqlContext: SQLContext): OrcRelation = { + def createEmpty( + pathString: String, + attributes: Seq[Attribute], + allowExisting: Boolean, + conf: Configuration, + sqlContext: SQLContext): OrcRelation = { val path = checkPath(pathString, allowExisting, conf) - /** set compression kind in hive 0.13.1 - * conf.set( - * HiveConf.ConfVars.OHIVE_ORC_DEFAULT_COMPRESS.varname, - * shortOrcCompressionCodecNames.getOrElse( - * sqlContext.orcCompressionCodec.toUpperCase, CompressionKind.NONE).name) - */ + /** set compression kind in hive 0.13.1 + * conf.set( + * HiveConf.ConfVars.OHIVE_ORC_DEFAULT_COMPRESS.varname, + * shortOrcCompressionCodecNames.getOrElse( + * sqlContext.orcCompressionCodec.toUpperCase, CompressionKind.NONE).name) + */ val orcRelation = new OrcRelation(path.toString, Some(conf), sqlContext) orcRelation @@ -219,20 +214,11 @@ private[sql] object OrcRelation { } private[sql] object OrcFileOperator { - final val COMPRESSION: String = "orcfiles.compression" - - /** - * - * @param origPath - * @return - */ - def readMetaData(origPath: Path): Reader = { - val job = new Job() - val conf = ContextUtil.getConfiguration(job).asInstanceOf[JobConf] + def readMetaData(origPath: Path, configuration: Option[Configuration]): Reader = { + val conf = configuration.getOrElse(new Configuration()) val fs: FileSystem = origPath.getFileSystem(conf) val orcFiles = FileSystemHelper.listFiles(origPath, conf, ".orc") if (orcFiles != Seq.empty) { -// NewFileInputFormat.setInputPaths(conf, orcFiles(0))// why set inputpath here if (fs.exists(origPath)) { OrcFile.createReader(fs, orcFiles(0)) } else { @@ -260,4 +246,3 @@ private[sql] object OrcFileOperator { } } } - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcTableOperations.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala similarity index 71% rename from sql/core/src/main/scala/org/apache/spark/sql/orc/OrcTableOperations.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala index bb1f342cc9ee..b7b6740aa060 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/orc/OrcTableOperations.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala @@ -16,40 +16,40 @@ */ -package org.apache.spark.sql.orc +package org.apache.spark.sql.hive.orc -import org.apache.spark.sql.execution.{ExistingRdd, LeafNode, UnaryNode, SparkPlan} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.{TaskContext, SerializableWritable} -import org.apache.spark.rdd.RDD +import java.io.IOException +import java.text.SimpleDateFormat +import java.util.{Locale, Date} +import scala.collection.JavaConversions._ -import _root_.parquet.hadoop.util.ContextUtil import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, FileOutputCommitter} import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.io.{Writable, NullWritable} import org.apache.hadoop.mapreduce.{TaskID, TaskAttemptContext, Job} - -import org.apache.hadoop.hive.ql.io.orc.{OrcFile, OrcSerde, OrcInputFormat, OrcOutputFormat} +import org.apache.hadoop.hive.ql.io.orc.{OrcSerde, OrcInputFormat, OrcOutputFormat} import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.ColumnProjectionUtils import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} - -import java.io.IOException -import java.text.SimpleDateFormat -import java.util.{Locale, Date} -import scala.collection.JavaConversions._ import org.apache.hadoop.mapred.{SparkHadoopMapRedUtil, Reporter, JobConf} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.parquet.FileSystemHelper +import org.apache.spark.{TaskContext, SerializableWritable} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils._ + /** * orc table scan operator. Imports the file that backs the given - * [[org.apache.spark.sql.orc.OrcRelation]] as a ``RDD[Row]``. + * [[org.apache.spark.sql.hive.orc.OrcRelation]] as a ``RDD[Row]``. */ case class OrcTableScan( - output: Seq[Attribute], - relation: OrcRelation, - columnPruningPred: Option[Expression]) + output: Seq[Attribute], + relation: OrcRelation, + columnPruningPred: Option[Expression]) extends LeafNode { @transient @@ -98,7 +98,7 @@ case class OrcTableScan( val sc = sqlContext.sparkContext val job = new Job(sc.hadoopConfiguration) - val conf: Configuration = ContextUtil.getConfiguration(job) + val conf: Configuration = job.getConfiguration val fileList = FileSystemHelper.listFiles(relation.path, conf) // add all paths in the directory but skip "hidden" ones such @@ -106,14 +106,13 @@ case class OrcTableScan( for (path <- fileList if !path.getName.startsWith("_")) { FileInputFormat.addInputPath(job, path) } - val serialConf = sc.broadcast(new SerializableWritable(conf)) setColumnIds(output, relation, conf) val inputClass = classOf[OrcInputFormat].asInstanceOf[ Class[_ <: org.apache.hadoop.mapred.InputFormat[Void, Row]]] val rowRdd = productToRowRdd(sc.hadoopRDD[Void, Row]( - serialConf.value.value.asInstanceOf[JobConf], inputClass, classOf[Void], classOf[Row])) + conf.asInstanceOf[JobConf], inputClass, classOf[Void], classOf[Row])) rowRdd } @@ -153,16 +152,16 @@ case class OrcTableScan( } else { val bufferedIterator = iterator.buffered bufferedIterator.map {r => - val values = getFieldValue.map(_(r)) - new GenericRow(values.map { - case n: String if n.toLowerCase == "null" => "" - case varchar: HiveVarchar => varchar.getValue - case decimal: HiveDecimal => - BigDecimal(decimal.bigDecimalValue) - case null => "" - case other => other - - }.toArray) + val values = getFieldValue.map(_(r)) + new GenericRow(values.map { + case n: String if n.toLowerCase == "null" => "" + case varchar: HiveVarchar => varchar.getValue + case decimal: HiveDecimal => + BigDecimal(decimal.bigDecimalValue) + case null => "" + case other => other + + }.toArray) } } } @@ -180,10 +179,12 @@ case class OrcTableScan( } /** - * - * @param relation - * @param child - * @param overwrite + * Operator that acts as a sink for queries on RDDs and can be used to + * store the output inside a directory of ORC files. This operator + * is similar to Hive's INSERT INTO TABLE operation in the sense that + * one can choose to either overwrite or append to a directory. Note + * that consecutive insertions to the same table must have compatible + * (source) schemas. */ private[sql] case class InsertIntoOrcTable( relation: OrcRelation, @@ -193,16 +194,14 @@ private[sql] case class InsertIntoOrcTable( override def output = child.output - val intputClass: Class[_] = getInputClass + val inputClass = getInputClass.getName @transient val sc = sqlContext.sparkContext - val inspector = sc.broadcast(getInspector(intputClass)) - @transient lazy val orcSerde = initFieldInfo private def getInputClass: Class[_] = { - val existRdd = child.asInstanceOf[ExistingRdd] + val existRdd = child.asInstanceOf[PhysicalRDD] val productClass = existRdd.rdd.firstParent.elementClassTag.runtimeClass logInfo("productClass is " + productClass) val clazz = productClass @@ -231,7 +230,6 @@ private[sql] case class InsertIntoOrcTable( assert(childRdd != null) val job = new Job(sqlContext.sparkContext.hadoopConfiguration) - // TODO: move that to function in object val conf = job.getConfiguration val fspath = new Path(relation.path) @@ -248,11 +246,14 @@ private[sql] case class InsertIntoOrcTable( } } - val existRdd = child.asInstanceOf[ExistingRdd] + val existRdd = child.asInstanceOf[PhysicalRDD] val parentRdd = existRdd.rdd.firstParent[Product] - val writableRdd = parentRdd.map(obj => { - orcSerde.serialize(obj, inspector.value) - }) + val writableRdd = parentRdd.mapPartitions { iter => + val objSnspector = ObjectInspectorFactory.getReflectionObjectInspector( + getContextOrSparkClassLoader.loadClass(inputClass), + ObjectInspectorFactory.ObjectInspectorOptions.JAVA) + iter.map(obj => orcSerde.serialize(obj, objSnspector)) + } saveAsHadoopFile(writableRdd, relation.rowClass, relation.path, conf) @@ -266,10 +267,10 @@ private[sql] case class InsertIntoOrcTable( // .. then we could use the default one and could use [[MutablePair]] // instead of ``Tuple2`` private def saveAsHadoopFile( - rdd: RDD[Writable], - rowClass: Class[_], - path: String, - @transient conf: Configuration) { + rdd: RDD[Writable], + rowClass: Class[_], + path: String, + @transient conf: Configuration) { val job = new Job(conf) val keyType = classOf[Void] job.setOutputKeyClass(keyType) @@ -287,7 +288,7 @@ private[sql] case class InsertIntoOrcTable( 1 } else { FileSystemHelper - .findMaxTaskId(FileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1 + .findMaxTaskId(FileOutputFormat.getOutputPath(job).toString, job.getConfiguration, "orc") + 1 } def getWriter( @@ -346,73 +347,4 @@ private[sql] case class InsertIntoOrcTable( sc.runJob(rdd, writeShard _) workerAndComitter._1.commitJob(jobTaskContext) } - } - -/** - * - */ -private[orc] object FileSystemHelper { - /** - * - * @param pathStr - * @param conf - * @return - */ - def listFiles(pathStr: String, conf: Configuration): Seq[Path] = { - val origPath = new Path(pathStr) - val fs = origPath.getFileSystem(conf) - if (fs == null) { - throw new IllegalArgumentException( - s"OrcTableOperations: Path $origPath is incorrectly formatted") - } - val path = origPath.makeQualified(fs) - if (!fs.exists(path) || !fs.getFileStatus(path).isDir) { - Seq.empty - } else { - fs.listStatus(path).map(_.getPath) - } - } - - /** - * - * @param origPath - * @param conf - * @param extension - * @return - */ - def listFiles(origPath: Path, conf: Configuration, extension: String): Seq[Path] = { - val fs = origPath.getFileSystem(conf) - if (fs == null) { - throw new IllegalArgumentException( - s"OrcTableOperations: Path $origPath is incorrectly formatted") - } - val path = origPath.makeQualified(fs) - if (fs.exists(path) && fs.getFileStatus(path).isDir) { - fs.listStatus(path).map(_.getPath).filter(p => p.getName.endsWith(extension)) - } else { - Seq.empty - } - } - - /** - * Finds the maximum taskid in the output file names at the given path. - */ - def findMaxTaskId(pathStr: String, conf: Configuration): Int = { - val files = FileSystemHelper.listFiles(pathStr, conf) - // filename pattern is part-r-.orc - val nameP = new scala.util.matching.Regex( """part-r-(\d{1,}).orc""", "taskid") - val hiddenFileP = new scala.util.matching.Regex("_.*") - files.map(_.getName).map { - case nameP(taskid) => taskid.toInt - case hiddenFileP() => 0 - case other: String => { - sys.error("ERROR: attempting to append to set of Orc files and found file" + - s"that does not match name pattern: $other") - 0 - } - case _ => 0 - }.reduceLeft((a, b) => if (a < b) b else a) - } -} - diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala new file mode 100644 index 000000000000..3636f8a3e030 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala @@ -0,0 +1,74 @@ +/* + * 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.hive.orc + +import org.apache.spark.{SparkConf, SparkContext} +import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.spark.util.Utils +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.hive.test.TestHive._ + + +import java.io.File + +case class TestRDDEntry(key: Int, value: String) + +case class AllDataTypes( + stringField: String, + intField: Int, + longField: Long, + floatField: Float, + doubleField: Double, + shortField: Short, + byteField: Byte, + booleanField: Boolean) + +class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { + + test("Read/Write All Types") { + val tempDir = getTempFilePath("orcTest").getCanonicalPath + val range = (0 to 255) + val data = sparkContext.parallelize(range) + .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + + data.saveAsOrcFile(tempDir) + + checkAnswer( + TestHive.orcFile(tempDir), + data.toSchemaRDD.collect().toSeq) + + Utils.deleteRecursively(new File(tempDir)) + } + + test("Compression options for writing to a Orcfile") { + val tempDir = getTempFilePath("orcTest").getCanonicalPath + val rdd = TestHive.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + + // test default compression codec, now only support zlib + rdd.saveAsOrcFile(tempDir) + var actualCodec = OrcFileOperator.readMetaData(new Path(tempDir), Some(new Configuration())).getCompression.name + assert(actualCodec == "ZLIB") + + Utils.deleteRecursively(new File(tempDir)) + } +} From 1db30b1d9e9b24fb5bd0933855b65f99f8ae715b Mon Sep 17 00:00:00 2001 From: scwf Date: Sat, 4 Oct 2014 15:37:50 +0800 Subject: [PATCH 05/18] fix scala style --- .../org/apache/spark/sql/parquet/ParquetTableOperations.scala | 3 ++- .../org/apache/spark/sql/hive/orc/OrcTableOperations.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index eed212cc0819..8fe69bdc6378 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -283,7 +283,8 @@ case class InsertIntoParquetTable( 1 } else { FileSystemHelper - .findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration, "parquet") + 1 + .findMaxTaskId( + NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration, "parquet") + 1 } def writeShard(context: TaskContext, iter: Iterator[Row]): Int = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala index b7b6740aa060..4edb83bdbe1e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala @@ -288,7 +288,8 @@ private[sql] case class InsertIntoOrcTable( 1 } else { FileSystemHelper - .findMaxTaskId(FileOutputFormat.getOutputPath(job).toString, job.getConfiguration, "orc") + 1 + .findMaxTaskId( + FileOutputFormat.getOutputPath(job).toString, job.getConfiguration, "orc") + 1 } def getWriter( From 9529d6893c7c5cf9991193a2c78c65b3f67c0445 Mon Sep 17 00:00:00 2001 From: w00228970 Date: Wed, 15 Oct 2014 05:52:58 +0800 Subject: [PATCH 06/18] address comments_1 --- docs/sql-programming-guide.md | 123 ++++++++++++++++++ .../examples/sql/hive/HiveFromSpark.scala | 10 -- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../plans/logical/basicOperators.scala | 25 +--- .../org/apache/spark/sql/SchemaRDDLike.scala | 6 +- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../sql/parquet/ParquetTableOperations.scala | 16 +-- .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/HiveStrategies.scala | 7 +- .../sql/hive/logical/hiveOperators.scala | 27 ++++ .../spark/sql/hive/orc/OrcRelation.scala | 29 ++--- .../spark/sql/hive/orc/ORCQuerySuite.scala | 2 +- 12 files changed, 179 insertions(+), 75 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/logical/hiveOperators.scala diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 368c3d0008b0..368acf68fa6a 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -597,6 +597,129 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or +## ORC Files + +[ORC](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+ORC) is a columnar format that is supported in Hive, it provides a highly efficient way to store data on HDFS to speed up query performance. +Spark SQL provides support for both reading and writing ORC files that automatically preserves the schema +of the original data. + +### Loading Data Programmatically + +Using the data from the above example: + +
+ +
+ +{% highlight scala %} +// Use HiveContext to read or write ORC File. +val sqlContext = new HiveContext(sc) +import sqlContext._ +val people: RDD[Person] = ... // An RDD of case class objects, from the previous example. + +// The RDD is implicitly converted to a SchemaRDD by createSchemaRDD, allowing it to be stored using ORC. +rdd.registerTempTable("people") +rdd.saveAsOrcFile("people.orc") + +// Read in the ORC file created above. ORC files are self-describing so the schema is preserved. +// The result of loading a ORC file is also a SchemaRDD. +val orcFile = hiveContext.orcFile("pair.orc") + +//ORC files can also be registered as tables and then used in SQL statements. +orcFile.registerTempTable("orcFile") +val teenagers = sqlContext.sql("SELECT name FROM orcFile WHERE age >= 13 AND age <= 19") +teenagers.map(t => "Name: " + t(0)).collect().foreach(println) +{% endhighlight %} + +
+ +
+ +{% highlight java %} +// Use JavaHiveContext to read or write ORC File. +JavaHiveContext sqlContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc); +JavaSchemaRDD schemaPeople = ... // The JavaSchemaRDD from the previous example. + +// JavaSchemaRDDs can be saved as ORC files, maintaining the schema information. +schemaPeople.saveAsOrcFile("people.orc"); + +// Read in the ORC file created above. ORC files are self-describing so the schema is preserved. +// The result of loading a ORC file is also a JavaSchemaRDD. +JavaSchemaRDD orcFile = sqlContext.orcFile("people.orc"); + +// ORC files can also be registered as tables and then used in SQL statements. +orcFile.registerTempTable("orcFile"); +JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM orcFile WHERE age >= 13 AND age <= 19"); +List teenagerNames = teenagers.map(new Function() { + public String call(Row row) { + return "Name: " + row.getString(0); + } +}).collect(); +{% endhighlight %} + +
+ +
+ +{% highlight python %} +# Use HiveContext to read or write ORC File. +from pyspark.sql import HiveContext +sqlContext = HiveContext(sc) + +schemaPeople # The SchemaRDD from the previous example. + +# SchemaRDDs can be saved as ORC files, maintaining the schema information. +schemaPeople.saveAsOrcFile("people.orc") + +# Read in the ORC file created above. ORC files are self-describing so the schema is preserved. +# The result of loading a ORC file is also a SchemaRDD. +orcFile = sqlContext.orcFile("people.orc") + +# ORC files can also be registered as tables and then used in SQL statements. +orcFile.registerTempTable("orcFile"); +teenagers = sqlContext.sql("SELECT name FROM orcFile WHERE age >= 13 AND age <= 19") +teenNames = teenagers.map(lambda p: "Name: " + p.name) +for teenName in teenNames.collect(): + print teenName +{% endhighlight %} + +
+ +
+ +### Configuration + +Configuration of ORC can be done using the `setConf` method on HiveContext or by running +`SET key=value` commands using SQL. + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.sql.parquet.binaryAsStringfalse + Some other Parquet-producing systems, in particular Impala and older versions of Spark SQL, do + not differentiate between binary data and strings when writing out the Parquet schema. This + flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. +
spark.sql.parquet.cacheMetadatafalse + Turns on caching of Parquet schema metadata. Can speed up querying of static data. +
spark.sql.parquet.compression.codecsnappy + Sets the compression codec use when writing Parquet files. Acceptable values include: + uncompressed, snappy, gzip, lzo. +
+ ## JSON Datasets
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index ea958eddb8f8..e26f213e8afa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -62,16 +62,6 @@ object HiveFromSpark { println("Result of SELECT *:") sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) - // Write out an RDD as a orc file. - rdd.saveAsOrcFile("pair.orc") - - // Read in orc file. Orc files are self-describing so the schmema is preserved. - val orcFile = hiveContext.orcFile("pair.orc") - - // These files can also be registered as tables. - orcFile.registerTempTable("orcFile") - sql("SELECT * FROM records r JOIN orcFile s ON r.key = s.key").collect().foreach(println) - sc.stop() } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index deb622c39faf..5668f09650bf 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -267,7 +267,7 @@ package object dsl { object plans { // scalastyle:ignore implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions { - def writeToFile(path: String) = WriteToFile(path, logicalPlan) + def writeToFile(path: String) = WriteToPaquetFile(path, logicalPlan) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 6fd3caf75c5f..13027cb583fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -114,22 +114,6 @@ case class InsertIntoTable( } } -case class InsertIntoOrcTable( - table: LogicalPlan, - partition: Map[String, Option[String]], - child: LogicalPlan, - overwrite: Boolean) - extends LogicalPlan { - // The table being inserted into is a child for the purposes of transformations. - def children = table :: child :: Nil - def references = Set.empty - def output = child.output - - override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { - case (childAttr, tableAttr) => childAttr.dataType == tableAttr.dataType - } -} - case class CreateTableAsSelect( databaseName: Option[String], tableName: String, @@ -138,19 +122,12 @@ case class CreateTableAsSelect( override lazy val resolved = (databaseName != None && childrenResolved) } -case class WriteToFile( +case class WriteToPaquetFile( path: String, child: LogicalPlan) extends UnaryNode { override def output = child.output } -case class WriteToOrcFile( - path: String, - child: LogicalPlan) extends UnaryNode { - def references = Set.empty - def output = child.output -} - case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { override def output = child.output } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 248b7c0d5af2..59e49e22645c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -54,7 +54,7 @@ private[sql] trait SchemaRDDLike { @transient protected[spark] val logicalPlan: LogicalPlan = baseLogicalPlan match { // For various commands (like DDL) and queries with side effects, we force query optimization to // happen right away to let these side effects take place eagerly. - case _: Command | _: InsertIntoTable | _: CreateTableAsSelect |_: WriteToFile => + case _: Command | _: InsertIntoTable | _: CreateTableAsSelect |_: WriteToPaquetFile => LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) case _ => baseLogicalPlan @@ -73,11 +73,11 @@ private[sql] trait SchemaRDDLike { * @group schema */ def saveAsParquetFile(path: String): Unit = { - sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd + sqlContext.executePlan(WriteToPaquetFile(path, logicalPlan)).toRdd } /** - * Saves the contents of this `SchemaRDD` as a orc file, preserving the schema. Files that + * Saves the contents of this `SchemaRDD` as a ORC file, preserving the schema. Files that * are written out using this method can be read back in as a SchemaRDD using the `orcFile` * function. * Note: you can only use it in HiveContext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index cf93d5ad7b50..47d2b1a21621 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -195,8 +195,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object ParquetOperations extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - // TODO: need to support writing to other types of files. Unify the below code paths. - case logical.WriteToFile(path, child) => + case logical.WriteToPaquetFile(path, child) => val relation = ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) // Note: overwrite=false because otherwise the metadata we just created will be deleted diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 8fe69bdc6378..a973a3491e70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -512,11 +512,11 @@ private[sql] object FileSystemHelper { val fs = origPath.getFileSystem(conf) if (fs == null) { throw new IllegalArgumentException( - s"OrcTableOperations: Path $origPath is incorrectly formatted") + s"Path $origPath is incorrectly formatted") } val path = origPath.makeQualified(fs) if (fs.exists(path) && fs.getFileStatus(path).isDir) { - fs.listStatus(path).map(_.getPath).filter(p => p.getName.endsWith(extension)) + fs.globStatus(path).map(_.getPath).filter(p => p.getName.endsWith(extension)) } else { Seq.empty } @@ -526,17 +526,13 @@ private[sql] object FileSystemHelper { * Finds the maximum taskid in the output file names at the given path. */ def findMaxTaskId(pathStr: String, conf: Configuration, extension: String): Int = { - val files = FileSystemHelper.listFiles(pathStr, conf) // filename pattern is part-r-.$extension - val nameP = extension match { - case "parquet" => new scala.util.matching.Regex( """part-r-(\d{1,}).parquet""", "taskid") - case "orc" => new scala.util.matching.Regex( """part-r-(\d{1,}).orc""", "taskid") - case _ => - sys.error(s"ERROR: unsupported extension: $extension") - } + require(Seq("orc", "parquet").contains(extension), s"Unsupported extension: $extension") + val nameP = new scala.util.matching.Regex(s"""part-r-(\d{1,}).$extension""", "taskid") + val files = FileSystemHelper.listFiles(pathStr, conf) val hiddenFileP = new scala.util.matching.Regex("_.*") files.map(_.getName).map { - case nameP(taskid) => taskid.toInt + case nameP(taskid) => taskid.toInt r case hiddenFileP() => 0 case other: String => { sys.error(s"ERROR: attempting to append to set of $extension files and found file" + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index b959070e4764..761b3b3075c9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -122,7 +122,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } /** - * Loads a Orc file, returning the result as a [[SchemaRDD]]. + * Loads a ORC file, returning the result as a [[SchemaRDD]]. * * @group userf */ @@ -135,7 +135,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * This registered table can be used as the target of future `insertInto` operations. * * {{{ - * val sqlContext = new SQLContext(...) + * val sqlContext = new HiveContext(...) * import sqlContext._ * * case class Person(name: String, age: Int) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 14f63d2ecb0a..a9183714e10e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -22,13 +22,12 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ -import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ -import org.apache.spark.sql.hive.orc.{InsertIntoOrcTable, OrcTableScan, OrcRelation} +import org.apache.spark.sql.hive.orc.{InsertIntoOrcTable, OrcRelation, OrcTableScan} import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.{SQLContext, SchemaRDD} @@ -228,8 +227,8 @@ private[hive] trait HiveStrategies { case logical.WriteToOrcFile(path, child) => val relation = OrcRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) - InsertIntoOrcTable(relation, planLater(child), overwrite=true) :: Nil - case logical.InsertIntoOrcTable(table: OrcRelation, partition, child, overwrite) => + InsertIntoOrcTable(relation, planLater(child), overwrite = true) :: Nil + case logical.InsertIntoTable(table: OrcRelation, partition, child, overwrite) => InsertIntoOrcTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters, relation: OrcRelation) => // TODO: need to implement predict push down. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/logical/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/logical/hiveOperators.scala new file mode 100644 index 000000000000..f46d90065544 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/logical/hiveOperators.scala @@ -0,0 +1,27 @@ +/* + * 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.hive.logical + +import org.apache.spark.sql.catalyst.plans.logical.{UnaryNode, LogicalPlan} + +case class WriteToOrcFile( + path: String, + child: LogicalPlan) extends UnaryNode { + def references = Set.empty + def output = child.output +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 5b3460553aa7..8062dfe32337 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.hive.orc import java.util.Properties import java.io.IOException -import org.apache.hadoop.hive.ql.stats.StatsSetupConst - import scala.collection.mutable import org.apache.hadoop.fs.{FileSystem, Path} @@ -29,6 +27,7 @@ import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.ql.io.orc._ import org.apache.hadoop.hive.ql.io.orc.OrcProto.Type.Kind +import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.spark.sql.parquet.FileSystemHelper import org.apache.spark.sql.SQLContext @@ -38,7 +37,6 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.types._ - private[sql] case class OrcRelation( path: String, @transient conf: Option[Configuration], @@ -59,9 +57,9 @@ private[sql] case class OrcRelation( override lazy val statistics = Statistics(sizeInBytes = sqlContext.defaultSizeInBytes) - def orcSchema: Seq[Attribute] = { + private def orcSchema: Seq[Attribute] = { val origPath = new Path(path) - val reader = OrcFileOperator.readMetaData(origPath, conf) + val reader = OrcFileOperator.getMetaDataReader(origPath, conf) if (null != reader) { val inspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] @@ -74,8 +72,8 @@ private[sql] case class OrcRelation( val totalType = reader.getTypes.get(0) val keys = totalType.getFieldNamesList val types = totalType.getSubtypesList - log.info("field name is {}", keys) - log.info("types is {}", types) + log.info("field names are {}", keys) + log.info("types are {}", types) val colBuff = new StringBuilder val typeBuff = new StringBuilder @@ -214,26 +212,21 @@ private[sql] object OrcRelation { } private[sql] object OrcFileOperator { - def readMetaData(origPath: Path, configuration: Option[Configuration]): Reader = { + def getMetaDataReader(origPath: Path, configuration: Option[Configuration]): Reader = { val conf = configuration.getOrElse(new Configuration()) val fs: FileSystem = origPath.getFileSystem(conf) val orcFiles = FileSystemHelper.listFiles(origPath, conf, ".orc") - if (orcFiles != Seq.empty) { - if (fs.exists(origPath)) { - OrcFile.createReader(fs, orcFiles(0)) - } else { - null - } + require(orcFiles != Seq.empty, "orcFiles is empty") + if (fs.exists(origPath)) { + OrcFile.createReader(fs, orcFiles(0)) } else { - null + throw new IOException(s"File not found: $origPath") } } def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) { - if (origPath == null) { - throw new IllegalArgumentException("Unable to write Parquet metadata: path is null") - } + require(origPath != null, "Unable to write ORC metadata: path is null") val fs = origPath.getFileSystem(conf) if (fs == null) { throw new IllegalArgumentException( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala index 3636f8a3e030..7d8d642d2262 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala @@ -66,7 +66,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { // test default compression codec, now only support zlib rdd.saveAsOrcFile(tempDir) - var actualCodec = OrcFileOperator.readMetaData(new Path(tempDir), Some(new Configuration())).getCompression.name + var actualCodec = OrcFileOperator.getMetaDataReader(new Path(tempDir), Some(new Configuration())).getCompression.name assert(actualCodec == "ZLIB") Utils.deleteRecursively(new File(tempDir)) From 89421ef18117c7ffdd44eff93c9360be785cacea Mon Sep 17 00:00:00 2001 From: scwf Date: Tue, 14 Oct 2014 21:15:35 -0700 Subject: [PATCH 07/18] log.info -> logInfo --- .../scala/org/apache/spark/sql/hive/orc/OrcRelation.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 8062dfe32337..82a55928c6a5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -72,8 +72,8 @@ private[sql] case class OrcRelation( val totalType = reader.getTypes.get(0) val keys = totalType.getFieldNamesList val types = totalType.getSubtypesList - log.info("field names are {}", keys) - log.info("types are {}", types) + logInfo("field names are {}", keys) + logInfo("types are {}", types) val colBuff = new StringBuilder val typeBuff = new StringBuilder @@ -91,6 +91,7 @@ private[sql] case class OrcRelation( typeBuff.setLength(typeBuff.length - 1) prop.setProperty("columns", colBuff.toString()) prop.setProperty("columns.types", typeBuff.toString()) + logInfo(s"columns are ${colBuff}, columns.types are $typeBuff") val attributes = convertToAttributes(reader, keys, types) attributes } else { @@ -122,7 +123,7 @@ private[sql] case class OrcRelation( case Kind.DOUBLE => new AttributeReference(keys.get(i), DoubleType, false)() case _ => { - log.info("unsupported datatype") + logInfo("unsupported datatype") null } } From 40f9d680ed10ea7783ccd90eb9c8aa2915e3386b Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 16 Oct 2014 15:35:30 -0700 Subject: [PATCH 08/18] address comments --- .../org/apache/spark/sql/SchemaRDDLike.scala | 12 -- .../sql/parquet/ParquetTableOperations.scala | 2 +- .../spark/sql/hive/HiveStrategies.scala | 5 +- .../sql/hive/logical/hiveOperators.scala | 27 ----- .../spark/sql/hive/orc/OrcRelation.scala | 109 +++--------------- .../sql/hive/orc/OrcTableOperations.scala | 80 ++++++++++--- .../apache/spark/sql/hive/orc/package.scala | 74 ++++++++++++ 7 files changed, 155 insertions(+), 154 deletions(-) delete mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/logical/hiveOperators.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 59e49e22645c..5b42011e3574 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -76,18 +76,6 @@ private[sql] trait SchemaRDDLike { sqlContext.executePlan(WriteToPaquetFile(path, logicalPlan)).toRdd } - /** - * Saves the contents of this `SchemaRDD` as a ORC file, preserving the schema. Files that - * are written out using this method can be read back in as a SchemaRDD using the `orcFile` - * function. - * Note: you can only use it in HiveContext - * - * @group schema - */ - def saveAsOrcFile(path: String): Unit = { - sqlContext.executePlan(WriteToOrcFile(path, logicalPlan)).toRdd - } - /** * Registers this RDD as a temporary table using the given name. The lifetime of this temporary * table is tied to the [[SQLContext]] that was used to create this SchemaRDD. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 12a0b49f9a32..5aec62055592 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -540,7 +540,7 @@ private[sql] object FileSystemHelper { val files = FileSystemHelper.listFiles(pathStr, conf) val hiddenFileP = new scala.util.matching.Regex("_.*") files.map(_.getName).map { - case nameP(taskid) => taskid.toInt r + case nameP(taskid) => taskid.toInt case hiddenFileP() => 0 case other: String => { sys.error(s"ERROR: attempting to append to set of $extension files and found file" + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index d2ede12523b0..fdcd8aa58c92 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ +import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ -import org.apache.spark.sql.hive.orc.{InsertIntoOrcTable, OrcRelation, OrcTableScan} +import org.apache.spark.sql.hive.orc.{WriteToOrcFile, InsertIntoOrcTable, OrcRelation, OrcTableScan} import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.{SQLContext, SchemaRDD} @@ -224,7 +225,7 @@ private[hive] trait HiveStrategies { object OrcOperations extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.WriteToOrcFile(path, child) => + case WriteToOrcFile(path, child) => val relation = OrcRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) InsertIntoOrcTable(relation, planLater(child), overwrite = true) :: Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/logical/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/logical/hiveOperators.scala deleted file mode 100644 index f46d90065544..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/logical/hiveOperators.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.hive.logical - -import org.apache.spark.sql.catalyst.plans.logical.{UnaryNode, LogicalPlan} - -case class WriteToOrcFile( - path: String, - child: LogicalPlan) extends UnaryNode { - def references = Set.empty - def output = child.output -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 82a55928c6a5..c539c45efe42 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -26,16 +26,14 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.ql.io.orc._ -import org.apache.hadoop.hive.ql.io.orc.OrcProto.Type.Kind -import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.spark.sql.parquet.FileSystemHelper import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} import org.apache.spark.sql.catalyst.analysis.{UnresolvedException, MultiInstanceRelation} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.hive.HiveMetastoreTypes private[sql] case class OrcRelation( path: String, @@ -49,85 +47,20 @@ private[sql] case class OrcRelation( var rowClass: Class[_] = null - val fieldIdCache: mutable.Map[String, Int] = new mutable.HashMap[String, Int] - - val fieldNameTypeCache: mutable.Map[String, String] = new mutable.HashMap[String, String] - override val output = orcSchema + // TODO: use statistics in ORC file override lazy val statistics = Statistics(sizeInBytes = sqlContext.defaultSizeInBytes) private def orcSchema: Seq[Attribute] = { + // get the schema info through ORC Reader val origPath = new Path(path) val reader = OrcFileOperator.getMetaDataReader(origPath, conf) + val inspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] + // data types that is inspected by this inspector + val schema = inspector.getTypeName - if (null != reader) { - val inspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] - val fields = inspector.getAllStructFieldRefs - - if (fields.size() == 0) { - return Seq.empty - } - - val totalType = reader.getTypes.get(0) - val keys = totalType.getFieldNamesList - val types = totalType.getSubtypesList - logInfo("field names are {}", keys) - logInfo("types are {}", types) - - val colBuff = new StringBuilder - val typeBuff = new StringBuilder - for (i <- 0 until fields.size()) { - val fieldName = fields.get(i).getFieldName - val typeName = fields.get(i).getFieldObjectInspector.getTypeName - colBuff.append(fieldName) - fieldNameTypeCache.put(fieldName, typeName) - fieldIdCache.put(fieldName, i) - colBuff.append(",") - typeBuff.append(typeName) - typeBuff.append(":") - } - colBuff.setLength(colBuff.length - 1) - typeBuff.setLength(typeBuff.length - 1) - prop.setProperty("columns", colBuff.toString()) - prop.setProperty("columns.types", typeBuff.toString()) - logInfo(s"columns are ${colBuff}, columns.types are $typeBuff") - val attributes = convertToAttributes(reader, keys, types) - attributes - } else { - Seq.empty - } - } - - def convertToAttributes( - reader: Reader, - keys: java.util.List[String], - types: java.util.List[Integer]): Seq[Attribute] = { - val range = 0.until(keys.size()) - range.map { - i => reader.getTypes.get(types.get(i)).getKind match { - case Kind.BOOLEAN => - new AttributeReference(keys.get(i), BooleanType, false)() - case Kind.STRING => - new AttributeReference(keys.get(i), StringType, true)() - case Kind.BYTE => - new AttributeReference(keys.get(i), ByteType, true)() - case Kind.SHORT => - new AttributeReference(keys.get(i), ShortType, true)() - case Kind.INT => - new AttributeReference(keys.get(i), IntegerType, true)() - case Kind.LONG => - new AttributeReference(keys.get(i), LongType, false)() - case Kind.FLOAT => - new AttributeReference(keys.get(i), FloatType, false)() - case Kind.DOUBLE => - new AttributeReference(keys.get(i), DoubleType, false)() - case _ => { - logInfo("unsupported datatype") - null - } - } - } + HiveMetastoreTypes.toDataType(schema).asInstanceOf[StructType].toAttributes } override def newInstance() = OrcRelation(path, conf, sqlContext).asInstanceOf[this.type] @@ -174,38 +107,26 @@ private[sql] object OrcRelation { sqlContext: SQLContext): OrcRelation = { val path = checkPath(pathString, allowExisting, conf) - /** set compression kind in hive 0.13.1 + /** TODO: set compression kind in hive 0.13.1 * conf.set( * HiveConf.ConfVars.OHIVE_ORC_DEFAULT_COMPRESS.varname, * shortOrcCompressionCodecNames.getOrElse( * sqlContext.orcCompressionCodec.toUpperCase, CompressionKind.NONE).name) */ - val orcRelation = new OrcRelation(path.toString, Some(conf), sqlContext) - - orcRelation + new OrcRelation(path.toString, Some(conf), sqlContext) } private def checkPath(pathStr: String, allowExisting: Boolean, conf: Configuration): Path = { - if (pathStr == null) { - throw new IllegalArgumentException("Unable to create OrcRelation: path is null") - } + require(pathStr != null, "Unable to create OrcRelation: path is null") val origPath = new Path(pathStr) val fs = origPath.getFileSystem(conf) - if (fs == null) { - throw new IllegalArgumentException( - s"Unable to create OrcRelation: incorrectly formatted path $pathStr") - } + require(fs != null, s"Unable to create OrcRelation: incorrectly formatted path $pathStr") val path = origPath.makeQualified(fs) - if (!allowExisting && fs.exists(path)) { - sys.error(s"File $pathStr already exists.") + if (!allowExisting) { + require(!fs.exists(path), s"File $pathStr already exists.") } - - if (fs.exists(path) && - !fs.getFileStatus(path) - .getPermission - .getUserAction - .implies(FsAction.READ_WRITE)) { - throw new IOException( + if (fs.exists(path)) { + require(fs.getFileStatus(path).getPermission.getUserAction.implies(FsAction.READ_WRITE), s"Unable to create OrcRelation: path $path not read-writable") } path diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala index 4edb83bdbe1e..0e8c616fe67a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala @@ -41,6 +41,20 @@ import org.apache.spark.sql.parquet.FileSystemHelper import org.apache.spark.{TaskContext, SerializableWritable} import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils._ +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode => LogicalUnaryNode} +import org.apache.spark.sql.execution.UnaryNode +import org.apache.spark.sql.catalyst.types.StructType +import org.apache.spark.sql.hive.HiveMetastoreTypes +import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfoUtils, TypeInfo} + +/** + * logical plan of writing to ORC file + */ +case class WriteToOrcFile( + path: String, + child: LogicalPlan) extends LogicalUnaryNode { + def output = child.output +} /** * orc table scan operator. Imports the file that backs the given @@ -107,7 +121,7 @@ case class OrcTableScan( FileInputFormat.addInputPath(job, path) } - setColumnIds(output, relation, conf) + addColumnIds(output, relation, conf) val inputClass = classOf[OrcInputFormat].asInstanceOf[ Class[_ <: org.apache.hadoop.mapred.InputFormat[Void, Row]]] @@ -117,34 +131,64 @@ case class OrcTableScan( } /** + * add column ids and names * @param output * @param relation * @param conf */ - def setColumnIds(output: Seq[Attribute], relation: OrcRelation, conf: Configuration) { - val idBuff = new StringBuilder() + def addColumnIds(output: Seq[Attribute], relation: OrcRelation, conf: Configuration) { + val fieldIdMap = relation.output.zipWithIndex.toMap - output.map(att => { + val ids = output.map(att => { val realName = att.name.toLowerCase(Locale.ENGLISH) - val id = relation.fieldIdCache.getOrElse(realName, null) - if (null != id) { - idBuff.append(id) - idBuff.append(",") + fieldIdMap.getOrElse(realName, -1) + }).filter(_ >= 0) + if (ids != null && !ids.isEmpty) { + ColumnProjectionUtils.appendReadColumnIDs(conf, ids) + } + + val names = output.map(_.name) + if (names != null && !names.isEmpty) { + ColumnProjectionUtils.appendReadColumnNames(conf, names) + } + } + + // Transform all given raw `Writable`s into `Row`s. + def fillObject( + iterator: scala.collection.Iterator[Writable], + nonPartitionKeyAttrs: Seq[(Attribute, Int)], + mutableRow: MutableRow): Iterator[Row] = { + val schema = StructType.fromAttributes(relation.output) + val orcSchema = HiveMetastoreTypes.toMetastoreType(schema) + val deserializer = new OrcSerde + val typeInfo: TypeInfo = TypeInfoUtils.getTypeInfoFromTypeString(orcSchema) + val soi = relation.getObjectInspector + + val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { + case (attr, ordinal) => + soi.getStructFieldRef(attr.name) -> ordinal + }.unzip + + val unwrappers = HadoopTypeConverter.unwrappers(fieldRefs) + // Map each tuple to a row object + iterator.map { value => + val raw = deserializer.deserialize(value) + var i = 0 + while (i < fieldRefs.length) { + val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) + if (fieldValue == null) { + mutableRow.setNullAt(fieldOrdinals(i)) + } else { + unwrappers(i)(fieldValue, mutableRow, fieldOrdinals(i)) + } + i += 1 } - }) - if (idBuff.length > 0) { - idBuff.setLength(idBuff.length - 1) + mutableRow: Row } - conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, idBuff.toString()) } - /** - * - * @param data - * @tparam A - * @return - */ def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { + val mutableRow = new SpecificMutableRow(output.map(_.dataType)) data.mapPartitions { iterator => if (iterator.isEmpty) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala new file mode 100644 index 000000000000..254a1b059390 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala @@ -0,0 +1,74 @@ +/* + * 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.hive + +import com.esotericsoftware.kryo.io.Output +import com.esotericsoftware.kryo.Kryo +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.SchemaRDD +import scala.reflect.runtime.universe.TypeTag + +package object orc { +// implicit class OrcContext(sqlContext: HiveContext) { +// +// def orcFile(filePath: String) = new SchemaRDD(sqlContext, +// OrcRelation(filePath, +// Some(sqlContext.sparkContext.hadoopConfiguration), sqlContext)) +// +// def createOrcFile[A <: Product : TypeTag]( +// path: String, +// allowExisting: Boolean = true, +// conf: Configuration = new Configuration()): SchemaRDD = { +// new SchemaRDD( +// sqlContext, +// OrcRelation.createEmpty(path, +// ScalaReflection.attributesFor[A], allowExisting, conf, sqlContext)) +// } +// } + + implicit class OrcSchemaRDD(rdd: SchemaRDD) { + /** + * Saves the contents of this `SchemaRDD` as a ORC file, preserving the schema. Files that + * are written out using this method can be read back in as a SchemaRDD using the `orcFile` + * function. + * Note: you can only use it in HiveContext + * + * @group schema + */ + def saveAsOrcFile(path: String): Unit = { + rdd.sqlContext.executePlan(WriteToOrcFile(path, + rdd.logicalPlan)).toRdd + } + } + + // for orc compression type, only take effect in hive 0.13.1 + val orcDefaultCompressVar = "hive.exec.orc.default.compress" + // for prediction push down in hive-0.13.1, don't enable it + val ORC_FILTER_PUSHDOWN_ENABLED = false + val SARG_PUSHDOWN = "sarg.pushdown"; + + def toKryo(input: SearchArgument) = { + val out = new Output(4 * 1024, 10 * 1024 * 1024); + new Kryo().writeObject(out, input); + out.close(); + Base64.encodeBase64String(out.toBytes()); + } +} \ No newline at end of file From 349bb92f04f16679bdf28a75f90349dfbe7d3123 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 16 Oct 2014 17:06:24 -0700 Subject: [PATCH 09/18] reuse TableReader --- .../spark/sql/hive/orc/OrcRelation.scala | 11 +- .../sql/hive/orc/OrcTableOperations.scala | 114 +++--------------- 2 files changed, 24 insertions(+), 101 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index c539c45efe42..36657db0db63 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.orc import java.util.Properties import java.io.IOException -import scala.collection.mutable import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.conf.Configuration @@ -35,6 +34,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.hive.HiveMetastoreTypes +import scala.collection.JavaConversions._ + private[sql] case class OrcRelation( path: String, @transient conf: Option[Configuration], @@ -60,6 +61,14 @@ private[sql] case class OrcRelation( // data types that is inspected by this inspector val schema = inspector.getTypeName + // set prop here, initial OrcSerde need it + val fields = inspector.getAllStructFieldRefs + val (columns, columnTypes) = fields.map { f => + f.getFieldName -> f.getFieldObjectInspector.getTypeName + }.unzip + prop.setProperty("columns", columns.mkString(",")) + prop.setProperty("columns.types", columnTypes.mkString(":")) + HiveMetastoreTypes.toDataType(schema).asInstanceOf[StructType].toAttributes } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala index 0e8c616fe67a..9cb7ec2e7830 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala @@ -32,7 +32,6 @@ import org.apache.hadoop.mapreduce.{TaskID, TaskAttemptContext, Job} import org.apache.hadoop.hive.ql.io.orc.{OrcSerde, OrcInputFormat, OrcOutputFormat} import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.ColumnProjectionUtils -import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.mapred.{SparkHadoopMapRedUtil, Reporter, JobConf} import org.apache.spark.sql.execution._ @@ -43,9 +42,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode => LogicalUnaryNode} import org.apache.spark.sql.execution.UnaryNode -import org.apache.spark.sql.catalyst.types.StructType -import org.apache.spark.sql.hive.HiveMetastoreTypes -import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfoUtils, TypeInfo} +import org.apache.spark.sql.hive.HadoopTableReader + +import scala.collection.JavaConversions._ /** * logical plan of writing to ORC file @@ -69,45 +68,12 @@ case class OrcTableScan( @transient lazy val serde: OrcSerde = initSerde - @transient - lazy val getFieldValue: Seq[Product => Any] = { - val inspector = serde.getObjectInspector.asInstanceOf[StructObjectInspector] - output.map(attr => { - val ref = inspector.getStructFieldRef(attr.name.toLowerCase(Locale.ENGLISH)) - row: Product => { - val fieldData = row.productElement(1) - val data = inspector.getStructFieldData(fieldData, ref) - unwrapData(data, ref.getFieldObjectInspector) - } - }) - } - private def initSerde(): OrcSerde = { val serde = new OrcSerde serde.initialize(null, relation.prop) serde } - def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { - case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) - case li: ListObjectInspector => - Option(li.getList(data)) - .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) - .orNull - case mi: MapObjectInspector => - Option(mi.getMap(data)).map( - _.map { - case (k, v) => - (unwrapData(k, mi.getMapKeyObjectInspector), - unwrapData(v, mi.getMapValueObjectInspector)) - }.toMap).orNull - case si: StructObjectInspector => - val allRefs = si.getAllStructFieldRefs - new GenericRow( - allRefs.map(r => - unwrapData(si.getStructFieldData(data, r), r.getFieldObjectInspector)).toArray) - } - override def execute(): RDD[Row] = { val sc = sqlContext.sparkContext val job = new Job(sc.hadoopConfiguration) @@ -123,10 +89,16 @@ case class OrcTableScan( addColumnIds(output, relation, conf) val inputClass = classOf[OrcInputFormat].asInstanceOf[ - Class[_ <: org.apache.hadoop.mapred.InputFormat[Void, Row]]] + Class[_ <: org.apache.hadoop.mapred.InputFormat[NullWritable, Writable]]] - val rowRdd = productToRowRdd(sc.hadoopRDD[Void, Row]( - conf.asInstanceOf[JobConf], inputClass, classOf[Void], classOf[Row])) + // use SpecificMutableRow to decrease GC garbage + val mutableRow = new SpecificMutableRow(output.map(_.dataType)) + val attrsWithIndex = output.zipWithIndex + val rowRdd = sc.hadoopRDD[NullWritable, Writable](conf.asInstanceOf[JobConf], inputClass, + classOf[NullWritable], classOf[Writable]).map(_._2).mapPartitions { iter => + val deserializer = serde + HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow) + } rowRdd } @@ -137,14 +109,14 @@ case class OrcTableScan( * @param conf */ def addColumnIds(output: Seq[Attribute], relation: OrcRelation, conf: Configuration) { - val fieldIdMap = relation.output.zipWithIndex.toMap + val fieldIdMap = relation.output.map(_.name).zipWithIndex.toMap val ids = output.map(att => { val realName = att.name.toLowerCase(Locale.ENGLISH) fieldIdMap.getOrElse(realName, -1) }).filter(_ >= 0) if (ids != null && !ids.isEmpty) { - ColumnProjectionUtils.appendReadColumnIDs(conf, ids) + ColumnProjectionUtils.appendReadColumnIDs(conf, ids.asInstanceOf[java.util.List[Integer]]) } val names = output.map(_.name) @@ -153,64 +125,6 @@ case class OrcTableScan( } } - // Transform all given raw `Writable`s into `Row`s. - def fillObject( - iterator: scala.collection.Iterator[Writable], - nonPartitionKeyAttrs: Seq[(Attribute, Int)], - mutableRow: MutableRow): Iterator[Row] = { - val schema = StructType.fromAttributes(relation.output) - val orcSchema = HiveMetastoreTypes.toMetastoreType(schema) - val deserializer = new OrcSerde - val typeInfo: TypeInfo = TypeInfoUtils.getTypeInfoFromTypeString(orcSchema) - val soi = relation.getObjectInspector - - val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { - case (attr, ordinal) => - soi.getStructFieldRef(attr.name) -> ordinal - }.unzip - - val unwrappers = HadoopTypeConverter.unwrappers(fieldRefs) - // Map each tuple to a row object - iterator.map { value => - val raw = deserializer.deserialize(value) - var i = 0 - while (i < fieldRefs.length) { - val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) - if (fieldValue == null) { - mutableRow.setNullAt(fieldOrdinals(i)) - } else { - unwrappers(i)(fieldValue, mutableRow, fieldOrdinals(i)) - } - i += 1 - } - mutableRow: Row - } - } - - def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { - val mutableRow = new SpecificMutableRow(output.map(_.dataType)) - data.mapPartitions { - iterator => - if (iterator.isEmpty) { - Iterator.empty - } else { - val bufferedIterator = iterator.buffered - bufferedIterator.map {r => - val values = getFieldValue.map(_(r)) - new GenericRow(values.map { - case n: String if n.toLowerCase == "null" => "" - case varchar: HiveVarchar => varchar.getValue - case decimal: HiveDecimal => - BigDecimal(decimal.bigDecimalValue) - case null => "" - case other => other - - }.toArray) - } - } - } - } - /** * Applies a (candidate) projection. * From 65483e80c2082c0e68a8862d87f8ac78def9cbce Mon Sep 17 00:00:00 2001 From: scwf Date: Fri, 17 Oct 2014 02:52:10 -0700 Subject: [PATCH 10/18] fix InsertIntoOrcTable --- .../sql/hive/orc/OrcTableOperations.scala | 89 +++++++++---------- 1 file changed, 40 insertions(+), 49 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala index 9cb7ec2e7830..1f42a1387dd1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala @@ -21,7 +21,6 @@ package org.apache.spark.sql.hive.orc import java.io.IOException import java.text.SimpleDateFormat import java.util.{Locale, Date} -import scala.collection.JavaConversions._ import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration @@ -39,12 +38,14 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.FileSystemHelper import org.apache.spark.{TaskContext, SerializableWritable} import org.apache.spark.rdd.RDD -import org.apache.spark.util.Utils._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode => LogicalUnaryNode} import org.apache.spark.sql.execution.UnaryNode -import org.apache.spark.sql.hive.HadoopTableReader +import org.apache.spark.sql.hive.{HiveMetastoreTypes, HadoopTableReader} import scala.collection.JavaConversions._ +import org.apache.spark.sql.catalyst.types.StructType +import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfoUtils, TypeInfo} +import org.apache.spark.sql.execution.PhysicalRDD /** * logical plan of writing to ORC file @@ -79,12 +80,12 @@ case class OrcTableScan( val job = new Job(sc.hadoopConfiguration) val conf: Configuration = job.getConfiguration - val fileList = FileSystemHelper.listFiles(relation.path, conf) - - // add all paths in the directory but skip "hidden" ones such - // as "_SUCCESS" - for (path <- fileList if !path.getName.startsWith("_")) { - FileInputFormat.addInputPath(job, path) + relation.path.split(",").foreach { curPath => + val qualifiedPath = { + val path = new Path(curPath) + path.getFileSystem(conf).makeQualified(path) + } + FileInputFormat.addInputPath(job, qualifiedPath) } addColumnIds(output, relation, conf) @@ -132,6 +133,7 @@ case class OrcTableScan( * @return Pruned TableScan. */ def pruneColumns(prunedAttributes: Seq[Attribute]): OrcTableScan = { + // Todo: prune projection OrcTableScan(prunedAttributes, relation, columnPruningPred) } } @@ -152,31 +154,13 @@ private[sql] case class InsertIntoOrcTable( override def output = child.output - val inputClass = getInputClass.getName - @transient val sc = sqlContext.sparkContext - @transient lazy val orcSerde = initFieldInfo + @transient lazy val orcSerde = initSerde - private def getInputClass: Class[_] = { - val existRdd = child.asInstanceOf[PhysicalRDD] - val productClass = existRdd.rdd.firstParent.elementClassTag.runtimeClass - logInfo("productClass is " + productClass) - val clazz = productClass - if (null == relation.rowClass) { - relation.rowClass = clazz - } - clazz - } - - private def getInspector(clazz: Class[_]): ObjectInspector = { - val inspector = ObjectInspectorFactory.getReflectionObjectInspector(clazz, - ObjectInspectorFactory.ObjectInspectorOptions.JAVA) - inspector - } - - private def initFieldInfo(): OrcSerde = { + private def initSerde(): OrcSerde = { val serde: OrcSerde = new OrcSerde + serde.initialize(null, relation.prop) serde } @@ -203,30 +187,36 @@ private[sql] case class InsertIntoOrcTable( + s" to InsertIntoOrcTable:\n${e.toString}") } } - - val existRdd = child.asInstanceOf[PhysicalRDD] - val parentRdd = existRdd.rdd.firstParent[Product] - val writableRdd = parentRdd.mapPartitions { iter => - val objSnspector = ObjectInspectorFactory.getReflectionObjectInspector( - getContextOrSparkClassLoader.loadClass(inputClass), - ObjectInspectorFactory.ObjectInspectorOptions.JAVA) - iter.map(obj => orcSerde.serialize(obj, objSnspector)) + val structType = StructType.fromAttributes(relation.output) + val orcSchema = HiveMetastoreTypes.toMetastoreType(structType) + + val writableRdd = childRdd.mapPartitions { iter => + val typeInfo: TypeInfo = + TypeInfoUtils.getTypeInfoFromTypeString(orcSchema) + val standardOI = TypeInfoUtils + .getStandardJavaObjectInspectorFromTypeInfo(typeInfo) + .asInstanceOf[StructObjectInspector] + val fieldOIs = standardOI + .getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray + val outputData = new Array[Any](fieldOIs.length) + iter.map { row => + var i = 0 + while (i < row.length) { + outputData(i) = HadoopTableReader.unwrapData(row(i), fieldOIs(i)) + i = 1 + } + orcSerde.serialize(outputData, standardOI) + } } - saveAsHadoopFile(writableRdd, relation.rowClass, relation.path, conf) + saveAsHadoopFile(writableRdd, relation.path, conf) // We return the child RDD to allow chaining (alternatively, one could return nothing). childRdd } - - // based on ``saveAsNewAPIHadoopFile`` in [[PairRDDFunctions]] - // TODO: Maybe PairRDDFunctions should use Product2 instead of Tuple2? - // .. then we could use the default one and could use [[MutablePair]] - // instead of ``Tuple2`` private def saveAsHadoopFile( rdd: RDD[Writable], - rowClass: Class[_], path: String, @transient conf: Configuration) { val job = new Job(conf) @@ -251,11 +241,12 @@ private[sql] case class InsertIntoOrcTable( } def getWriter( - outFormat: OrcOutputFormat, - conf: Configuration, - path: Path, - reporter: Reporter) = { + outFormat: OrcOutputFormat, + conf: Configuration, + path: Path, + reporter: Reporter) = { val fs = path.getFileSystem(conf) + outFormat.getRecordWriter(fs, conf.asInstanceOf[JobConf], path.toUri.getPath, reporter). asInstanceOf[org.apache.hadoop.mapred.RecordWriter[NullWritable, Writable]] } From 629f95ef0f26d2578cff1491b1d4e493243a53d4 Mon Sep 17 00:00:00 2001 From: scwf Date: Fri, 17 Oct 2014 22:15:45 -0700 Subject: [PATCH 11/18] address comments --- .../scala/org/apache/spark/util/Utils.scala | 12 ++- .../org/apache/spark/sql/SQLContext.scala | 6 +- .../sql/parquet/ParquetTableOperations.scala | 4 +- .../apache/spark/sql/hive/HiveContext.scala | 99 ++++++++++++------- .../spark/sql/hive/orc/OrcRelation.scala | 61 +++--------- .../sql/hive/orc/OrcTableOperations.scala | 8 +- .../apache/spark/sql/hive/orc/package.scala | 28 ++---- .../spark/sql/hive/orc/ORCQuerySuite.scala | 11 ++- 8 files changed, 109 insertions(+), 120 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 53a7512edd85..03a484a6cb9d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -659,11 +659,15 @@ private[spark] object Utils extends Logging { } private def listFilesSafely(file: File): Seq[File] = { - val files = file.listFiles() - if (files == null) { - throw new IOException("Failed to list files for dir: " + file) + if (file.exists()) { + val files = file.listFiles() + if (files == null) { + throw new IOException("Failed to list files for dir: " + file) + } + files + } else { + List() } - files } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 23e7b2d27077..0a0e351c8ac8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -149,8 +149,10 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - def parquetFile(path: String): SchemaRDD = - new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) + def parquetFile(path: String): SchemaRDD = { + val a = new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) + a + } /** * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 5aec62055592..fb42c34c29ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -524,7 +524,7 @@ private[sql] object FileSystemHelper { } val path = origPath.makeQualified(fs) if (fs.exists(path) && fs.getFileStatus(path).isDir) { - fs.globStatus(path).map(_.getPath).filter(p => p.getName.endsWith(extension)) + fs.listStatus(path).map(_.getPath).filter(p => p.getName.endsWith(extension)) } else { Seq.empty } @@ -536,7 +536,7 @@ private[sql] object FileSystemHelper { def findMaxTaskId(pathStr: String, conf: Configuration, extension: String): Int = { // filename pattern is part-r-.$extension require(Seq("orc", "parquet").contains(extension), s"Unsupported extension: $extension") - val nameP = new scala.util.matching.Regex(s"""part-r-(\d{1,}).$extension""", "taskid") + val nameP = new scala.util.matching.Regex(s"""part-r-(\\d{1,}).$extension""", "taskid") val files = FileSystemHelper.listFiles(pathStr, conf) val hiddenFileP = new scala.util.matching.Regex("_.*") files.map(_.getName).map { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 67c79bc4c853..d1cbe6085376 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -40,14 +40,20 @@ import org.apache.hadoop.hive.serde2.io.DateWritable import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.hive.orc.OrcRelation +import org.apache.spark.sql.hive.orc.{OrcSchemaRDD, OrcRelation} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators} import org.apache.spark.sql.catalyst.analysis.{OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.{Command => PhysicalCommand, ExtractPythonUdfs, QueryExecutionException} +import org.apache.spark.sql.execution.{Command => PhysicalCommand, _} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand +import org.apache.spark.sql.catalyst.plans.logical.SetCommand +import scala.Some +import org.apache.spark.sql.catalyst.plans.logical.NativeCommand +import org.apache.spark.sql.hive.MetastoreRelation +import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand +import org.apache.spark.sql.execution.LogicalRDD /** * DEPRECATED: Use HiveContext instead. @@ -110,6 +116,25 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { @deprecated("hql() is deprecated as the sql function now parses using HiveQL by default. " + s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1") def hql(hqlQuery: String): SchemaRDD = hiveql(hqlQuery) + /** + * Creates a SchemaRDD from an RDD of case classes. + * + * @group userf + */ +// implicit def createOrcSchemaRDD(rdd: SchemaRDD) = { +// new OrcSchemaRDD(rdd) +// } + + /** + * Creates a SchemaRDD from an RDD of case classes. + * + * @group userf + */ + implicit override def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = { + SparkPlan.currentContext.set(self) + new OrcSchemaRDD(this, + LogicalRDD(ScalaReflection.attributesFor[A], RDDConversions.productToRowRdd(rdd))(self)) + } /** * Creates a table using the schema of the given class. @@ -128,41 +153,41 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * @group userf */ def orcFile(path: String): SchemaRDD = - new SchemaRDD(this, orc.OrcRelation(path, Some(sparkContext.hadoopConfiguration), this)) - - /** - * :: Experimental :: - * Creates an empty orc file with the schema of class `A`, which can be registered as a table. - * This registered table can be used as the target of future `insertInto` operations. - * - * {{{ - * val sqlContext = new HiveContext(...) - * import sqlContext._ - * - * case class Person(name: String, age: Int) - * createOrcFile[Person]("path/to/file.orc").registerTempTable("people") - * sql("INSERT INTO people SELECT 'michael', 29") - * }}} - * - * @tparam A A case class type that describes the desired schema of the orc file to be - * created. - * @param path The path where the directory containing parquet metadata should be created. - * Data inserted into this table will also be stored at this location. - * @param allowExisting When false, an exception will be thrown if this directory already exists. - * @param conf A Hadoop configuration object that can be used to specify options to the parquet - * output format. - * - * @group userf - */ - @Experimental - def createOrcFile[A <: Product : TypeTag]( - path: String, - allowExisting: Boolean = true, - conf: Configuration = new Configuration()): SchemaRDD = { - new SchemaRDD( - this, - OrcRelation.createEmpty(path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) - } + new SchemaRDD(this, orc.OrcRelation(Seq.empty, path, Some(sparkContext.hadoopConfiguration), this)) + +// /** +// * :: Experimental :: +// * Creates an empty orc file with the schema of class `A`, which can be registered as a table. +// * This registered table can be used as the target of future `insertInto` operations. +// * +// * {{{ +// * val sqlContext = new HiveContext(...) +// * import sqlContext._ +// * +// * case class Person(name: String, age: Int) +// * createOrcFile[Person]("path/to/file.orc").registerTempTable("people") +// * sql("INSERT INTO people SELECT 'michael', 29") +// * }}} +// * +// * @tparam A A case class type that describes the desired schema of the orc file to be +// * created. +// * @param path The path where the directory containing parquet metadata should be created. +// * Data inserted into this table will also be stored at this location. +// * @param allowExisting When false, an exception will be thrown if this directory already exists. +// * @param conf A Hadoop configuration object that can be used to specify options to the parquet +// * output format. +// * +// * @group userf +// */ +// @Experimental +// def createOrcFile[A <: Product : TypeTag]( +// path: String, +// allowExisting: Boolean = true, +// conf: Configuration = new Configuration()): SchemaRDD = { +// new SchemaRDD( +// this, +// OrcRelation.createEmpty(path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) +// } /** * Analyzes the given table in the current database to generate statistics, which will be diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 36657db0db63..20e82e874690 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.hive.HiveMetastoreTypes import scala.collection.JavaConversions._ private[sql] case class OrcRelation( + attributes: Seq[Attribute], path: String, @transient conf: Option[Configuration], @transient sqlContext: SQLContext, @@ -46,9 +47,7 @@ private[sql] case class OrcRelation( val prop: Properties = new Properties - var rowClass: Class[_] = null - - override val output = orcSchema + override lazy val output = attributes ++ orcSchema // TODO: use statistics in ORC file override lazy val statistics = Statistics(sizeInBytes = sqlContext.defaultSizeInBytes) @@ -57,6 +56,10 @@ private[sql] case class OrcRelation( // get the schema info through ORC Reader val origPath = new Path(path) val reader = OrcFileOperator.getMetaDataReader(origPath, conf) + if (reader == null) { + // return empty seq when saveAsOrcFile + return Seq.empty + } val inspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] // data types that is inspected by this inspector val schema = inspector.getTypeName @@ -72,7 +75,8 @@ private[sql] case class OrcRelation( HiveMetastoreTypes.toDataType(schema).asInstanceOf[StructType].toAttributes } - override def newInstance() = OrcRelation(path, conf, sqlContext).asInstanceOf[this.type] + override def newInstance() = + OrcRelation(attributes, path, conf, sqlContext).asInstanceOf[this.type] } private[sql] object OrcRelation { @@ -96,33 +100,8 @@ private[sql] object OrcRelation { child, "Attempt to create Orc table from unresolved child") } - createEmpty(pathString, child.output, false, conf, sqlContext) - } - - /** - * Creates an empty OrcRelation and underlying Orcfile that only - * consists of the Metadata for the given schema. - * - * @param pathString The directory the Orcfile will be stored in. - * @param attributes The schema of the relation. - * @param conf A configuration to be used. - * @return An empty OrcRelation. - */ - def createEmpty( - pathString: String, - attributes: Seq[Attribute], - allowExisting: Boolean, - conf: Configuration, - sqlContext: SQLContext): OrcRelation = { - val path = checkPath(pathString, allowExisting, conf) - - /** TODO: set compression kind in hive 0.13.1 - * conf.set( - * HiveConf.ConfVars.OHIVE_ORC_DEFAULT_COMPRESS.varname, - * shortOrcCompressionCodecNames.getOrElse( - * sqlContext.orcCompressionCodec.toUpperCase, CompressionKind.NONE).name) - */ - new OrcRelation(path.toString, Some(conf), sqlContext) + val path = checkPath(pathString, false, conf) + new OrcRelation(child.output, path.toString, Some(conf), sqlContext) } private def checkPath(pathStr: String, allowExisting: Boolean, conf: Configuration): Path = { @@ -147,26 +126,14 @@ private[sql] object OrcFileOperator { val conf = configuration.getOrElse(new Configuration()) val fs: FileSystem = origPath.getFileSystem(conf) val orcFiles = FileSystemHelper.listFiles(origPath, conf, ".orc") - require(orcFiles != Seq.empty, "orcFiles is empty") + if (orcFiles == Seq.empty) { + // should return null when write to orc file + return null + } if (fs.exists(origPath)) { OrcFile.createReader(fs, orcFiles(0)) } else { throw new IOException(s"File not found: $origPath") } } - - - def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) { - require(origPath != null, "Unable to write ORC metadata: path is null") - val fs = origPath.getFileSystem(conf) - if (fs == null) { - throw new IllegalArgumentException( - s"Unable to write Orc metadata: path $origPath is incorrectly formatted") - } - - val path = origPath.makeQualified(fs) - if (fs.exists(path) && !fs.getFileStatus(path).isDir) { - throw new IllegalArgumentException(s"Expected to write to directory $path but found file") - } - } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala index 1f42a1387dd1..a036355f6572 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala @@ -45,7 +45,6 @@ import org.apache.spark.sql.hive.{HiveMetastoreTypes, HadoopTableReader} import scala.collection.JavaConversions._ import org.apache.spark.sql.catalyst.types.StructType import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfoUtils, TypeInfo} -import org.apache.spark.sql.execution.PhysicalRDD /** * logical plan of writing to ORC file @@ -115,9 +114,9 @@ case class OrcTableScan( val ids = output.map(att => { val realName = att.name.toLowerCase(Locale.ENGLISH) fieldIdMap.getOrElse(realName, -1) - }).filter(_ >= 0) + }).filter(_ >= 0).map(_.asInstanceOf[Integer]) if (ids != null && !ids.isEmpty) { - ColumnProjectionUtils.appendReadColumnIDs(conf, ids.asInstanceOf[java.util.List[Integer]]) + ColumnProjectionUtils.appendReadColumnIDs(conf, ids) } val names = output.map(_.name) @@ -188,6 +187,7 @@ private[sql] case class InsertIntoOrcTable( } } val structType = StructType.fromAttributes(relation.output) + // get Type String to build typeInfo val orcSchema = HiveMetastoreTypes.toMetastoreType(structType) val writableRdd = childRdd.mapPartitions { iter => @@ -203,7 +203,7 @@ private[sql] case class InsertIntoOrcTable( var i = 0 while (i < row.length) { outputData(i) = HadoopTableReader.unwrapData(row(i), fieldOIs(i)) - i = 1 + i += 1 } orcSerde.serialize(outputData, standardOI) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala index 254a1b059390..e81db568ef43 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala @@ -23,28 +23,15 @@ import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.ql.io.sarg.SearchArgument import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.SchemaRDD +import org.apache.spark.sql.{SQLContext, SchemaRDD} import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan package object orc { -// implicit class OrcContext(sqlContext: HiveContext) { -// -// def orcFile(filePath: String) = new SchemaRDD(sqlContext, -// OrcRelation(filePath, -// Some(sqlContext.sparkContext.hadoopConfiguration), sqlContext)) -// -// def createOrcFile[A <: Product : TypeTag]( -// path: String, -// allowExisting: Boolean = true, -// conf: Configuration = new Configuration()): SchemaRDD = { -// new SchemaRDD( -// sqlContext, -// OrcRelation.createEmpty(path, -// ScalaReflection.attributesFor[A], allowExisting, conf, sqlContext)) -// } -// } - - implicit class OrcSchemaRDD(rdd: SchemaRDD) { + class OrcSchemaRDD( + @transient val sqlContext1: SQLContext, + @transient val baseLogicalPlan1: LogicalPlan) + extends SchemaRDD(sqlContext1, baseLogicalPlan1) { /** * Saves the contents of this `SchemaRDD` as a ORC file, preserving the schema. Files that * are written out using this method can be read back in as a SchemaRDD using the `orcFile` @@ -54,8 +41,7 @@ package object orc { * @group schema */ def saveAsOrcFile(path: String): Unit = { - rdd.sqlContext.executePlan(WriteToOrcFile(path, - rdd.logicalPlan)).toRdd + sqlContext.executePlan(WriteToOrcFile(path, logicalPlan)).toRdd } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala index 7d8d642d2262..743225bdf610 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala @@ -51,12 +51,11 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) data.saveAsOrcFile(tempDir) - checkAnswer( TestHive.orcFile(tempDir), data.toSchemaRDD.collect().toSeq) - Utils.deleteRecursively(new File(tempDir)) +// Utils.deleteRecursively(new File(tempDir)) } test("Compression options for writing to a Orcfile") { @@ -69,6 +68,12 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { var actualCodec = OrcFileOperator.getMetaDataReader(new Path(tempDir), Some(new Configuration())).getCompression.name assert(actualCodec == "ZLIB") - Utils.deleteRecursively(new File(tempDir)) +// Utils.deleteRecursively(new File(tempDir)) + } + + test("Orc metadata reader") { + val path = new Path("/private/var/folders/q7/whr53mh905l1xm_zqgk00j3m0000gn/T/orcTest5484580860941213910") + val reader = OrcFileOperator.getMetaDataReader(path, Some(TestHive.sparkContext.hadoopConfiguration)) + reader } } From d1bba2315d7aedf09707fe597329196ef1b7daeb Mon Sep 17 00:00:00 2001 From: scwf Date: Fri, 17 Oct 2014 23:19:00 -0700 Subject: [PATCH 12/18] added test for read orc schema --- .../sql/parquet/ParquetTableOperations.scala | 5 +- .../spark/sql/hive/orc/OrcRelation.scala | 58 +++++++++---------- .../apache/spark/sql/hive/orc/package.scala | 16 +---- .../spark/sql/hive/orc/ORCQuerySuite.scala | 30 ++++++---- 4 files changed, 51 insertions(+), 58 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index fb42c34c29ce..90b00bf9e26e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -497,7 +497,7 @@ private[parquet] object FilteringParquetRowInputFormat { .build[FileStatus, Array[BlockLocation]]() } -private[sql] object FileSystemHelper { +private[sql] object FileSystemHelper extends Logging{ def listFiles(pathStr: String, conf: Configuration): Seq[Path] = { val origPath = new Path(pathStr) val fs = origPath.getFileSystem(conf) @@ -523,9 +523,10 @@ private[sql] object FileSystemHelper { s"Path $origPath is incorrectly formatted") } val path = origPath.makeQualified(fs) - if (fs.exists(path) && fs.getFileStatus(path).isDir) { + if (fs.exists(path)) { fs.listStatus(path).map(_.getPath).filter(p => p.getName.endsWith(extension)) } else { + println(s"${path.toString} not exists") Seq.empty } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 20e82e874690..79d267adb14e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.hive.HiveMetastoreTypes import scala.collection.JavaConversions._ +import org.apache.spark.Logging private[sql] case class OrcRelation( attributes: Seq[Attribute], @@ -47,34 +48,11 @@ private[sql] case class OrcRelation( val prop: Properties = new Properties - override lazy val output = attributes ++ orcSchema + override lazy val output = attributes ++ OrcFileOperator.orcSchema(path, conf, prop) // TODO: use statistics in ORC file override lazy val statistics = Statistics(sizeInBytes = sqlContext.defaultSizeInBytes) - private def orcSchema: Seq[Attribute] = { - // get the schema info through ORC Reader - val origPath = new Path(path) - val reader = OrcFileOperator.getMetaDataReader(origPath, conf) - if (reader == null) { - // return empty seq when saveAsOrcFile - return Seq.empty - } - val inspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] - // data types that is inspected by this inspector - val schema = inspector.getTypeName - - // set prop here, initial OrcSerde need it - val fields = inspector.getAllStructFieldRefs - val (columns, columnTypes) = fields.map { f => - f.getFieldName -> f.getFieldObjectInspector.getTypeName - }.unzip - prop.setProperty("columns", columns.mkString(",")) - prop.setProperty("columns.types", columnTypes.mkString(":")) - - HiveMetastoreTypes.toDataType(schema).asInstanceOf[StructType].toAttributes - } - override def newInstance() = OrcRelation(attributes, path, conf, sqlContext).asInstanceOf[this.type] } @@ -121,7 +99,7 @@ private[sql] object OrcRelation { } } -private[sql] object OrcFileOperator { +private[sql] object OrcFileOperator{ def getMetaDataReader(origPath: Path, configuration: Option[Configuration]): Reader = { val conf = configuration.getOrElse(new Configuration()) val fs: FileSystem = origPath.getFileSystem(conf) @@ -130,10 +108,32 @@ private[sql] object OrcFileOperator { // should return null when write to orc file return null } - if (fs.exists(origPath)) { - OrcFile.createReader(fs, orcFiles(0)) - } else { - throw new IOException(s"File not found: $origPath") + OrcFile.createReader(fs, orcFiles(0)) + } + + def orcSchema( + path: String, + conf: Option[Configuration], + prop: Properties): Seq[Attribute] = { + // get the schema info through ORC Reader + val origPath = new Path(path) + val reader = getMetaDataReader(origPath, conf) + if (reader == null) { + // return empty seq when saveAsOrcFile + return Seq.empty } + val inspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] + // data types that is inspected by this inspector + val schema = inspector.getTypeName + + // set prop here, initial OrcSerde need it + val fields = inspector.getAllStructFieldRefs + val (columns, columnTypes) = fields.map { f => + f.getFieldName -> f.getFieldObjectInspector.getTypeName + }.unzip + prop.setProperty("columns", columns.mkString(",")) + prop.setProperty("columns.types", columnTypes.mkString(":")) + + HiveMetastoreTypes.toDataType(schema).asInstanceOf[StructType].toAttributes } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala index e81db568ef43..06b2d0e3aee3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala @@ -17,14 +17,7 @@ package org.apache.spark.sql.hive -import com.esotericsoftware.kryo.io.Output -import com.esotericsoftware.kryo.Kryo -import org.apache.commons.codec.binary.Base64 -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.ql.io.sarg.SearchArgument -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.{SQLContext, SchemaRDD} -import scala.reflect.runtime.universe.TypeTag import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan package object orc { @@ -49,12 +42,5 @@ package object orc { val orcDefaultCompressVar = "hive.exec.orc.default.compress" // for prediction push down in hive-0.13.1, don't enable it val ORC_FILTER_PUSHDOWN_ENABLED = false - val SARG_PUSHDOWN = "sarg.pushdown"; - - def toKryo(input: SearchArgument) = { - val out = new Output(4 * 1024, 10 * 1024 * 1024); - new Kryo().writeObject(out, input); - out.close(); - Base64.encodeBase64String(out.toBytes()); - } + val SARG_PUSHDOWN = "sarg.pushdown" } \ No newline at end of file diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala index 743225bdf610..0d883bb9b400 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala @@ -17,17 +17,16 @@ package org.apache.spark.sql.hive.orc -import org.apache.spark.{SparkConf, SparkContext} -import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} +import java.util.Properties +import org.scalatest.BeforeAndAfterAll import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.util.Utils import org.apache.spark.sql.QueryTest import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.hive.test.TestHive._ - - +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.util.Utils import java.io.File case class TestRDDEntry(key: Int, value: String) @@ -55,7 +54,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { TestHive.orcFile(tempDir), data.toSchemaRDD.collect().toSeq) -// Utils.deleteRecursively(new File(tempDir)) + Utils.deleteRecursively(new File(tempDir)) } test("Compression options for writing to a Orcfile") { @@ -65,15 +64,22 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { // test default compression codec, now only support zlib rdd.saveAsOrcFile(tempDir) - var actualCodec = OrcFileOperator.getMetaDataReader(new Path(tempDir), Some(new Configuration())).getCompression.name + val actualCodec = OrcFileOperator.getMetaDataReader(new Path(tempDir), Some(new Configuration())).getCompression.name assert(actualCodec == "ZLIB") -// Utils.deleteRecursively(new File(tempDir)) + Utils.deleteRecursively(new File(tempDir)) } - test("Orc metadata reader") { - val path = new Path("/private/var/folders/q7/whr53mh905l1xm_zqgk00j3m0000gn/T/orcTest5484580860941213910") - val reader = OrcFileOperator.getMetaDataReader(path, Some(TestHive.sparkContext.hadoopConfiguration)) - reader + test("Get ORC Schema with ORC Reader") { + val path = "sql/hive/src/test/resources/data/files/orcfiles" + val attributes = OrcFileOperator.orcSchema(path, Some(TestHive.sparkContext.hadoopConfiguration), new Properties()) + assert(attributes(0).dataType == StringType) + assert(attributes(1).dataType == IntegerType) + assert(attributes(2).dataType == LongType) + assert(attributes(3).dataType == FloatType) + assert(attributes(4).dataType == DoubleType) + assert(attributes(5).dataType == ShortType) + assert(attributes(6).dataType == ByteType) + assert(attributes(7).dataType == BooleanType) } } From b46ecb1cf6336cd2b20b24d893bd660f43d51814 Mon Sep 17 00:00:00 2001 From: scwf Date: Fri, 17 Oct 2014 23:27:52 -0700 Subject: [PATCH 13/18] add test files --- .../data/files/orcfiles/.part-r-1.orc.crc | Bin 0 -> 20 bytes .../data/files/orcfiles/.part-r-2.orc.crc | Bin 0 -> 20 bytes .../test/resources/data/files/orcfiles/_SUCCESS | 0 .../resources/data/files/orcfiles/part-r-1.orc | Bin 0 -> 1296 bytes .../resources/data/files/orcfiles/part-r-2.orc | Bin 0 -> 1286 bytes 5 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 sql/hive/src/test/resources/data/files/orcfiles/.part-r-1.orc.crc create mode 100644 sql/hive/src/test/resources/data/files/orcfiles/.part-r-2.orc.crc create mode 100755 sql/hive/src/test/resources/data/files/orcfiles/_SUCCESS create mode 100755 sql/hive/src/test/resources/data/files/orcfiles/part-r-1.orc create mode 100755 sql/hive/src/test/resources/data/files/orcfiles/part-r-2.orc diff --git a/sql/hive/src/test/resources/data/files/orcfiles/.part-r-1.orc.crc b/sql/hive/src/test/resources/data/files/orcfiles/.part-r-1.orc.crc new file mode 100644 index 0000000000000000000000000000000000000000..048f7346c2c05e08ebe5a11aed78665f40e3e378 GIT binary patch literal 20 bcmYc;N@ieSU}BgvvBaS1*w4n6U^Z_6Ku`wu literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/data/files/orcfiles/.part-r-2.orc.crc b/sql/hive/src/test/resources/data/files/orcfiles/.part-r-2.orc.crc new file mode 100644 index 0000000000000000000000000000000000000000..52810863f2ae9a45c8f00f8d022cbe68369ea23b GIT binary patch literal 20 bcmYc;N@ieSU}BK>5DB}x^|_y9#fBvSHcbY& literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/data/files/orcfiles/_SUCCESS b/sql/hive/src/test/resources/data/files/orcfiles/_SUCCESS new file mode 100755 index 000000000000..e69de29bb2d1 diff --git a/sql/hive/src/test/resources/data/files/orcfiles/part-r-1.orc b/sql/hive/src/test/resources/data/files/orcfiles/part-r-1.orc new file mode 100755 index 0000000000000000000000000000000000000000..3188e14c1fa482780268cdff5acb8ce8c2e5609e GIT binary patch literal 1296 zcmeYda^`1X;9?bG=4fElVPJSH^fW1zfl*PekIhZBOwWy3QO(<%K^Le*go~MhfkB8D zs7{E3gF)aQqeMeJP6;cZQ9+NBk`e@h9%Lj;ne;5BfyvRJQsJJX(4iae85ju3<1`qj zgeuUD0$eOWH!=X-*TAU7%*E)Sz{nume~^*ck%K8ZD!}1)zE0ddr871gqR#2aF)X?u z(37yChUH{Zu*u;c0<&KjT#8`B< zTypCuvwET8BYyXbz{2TvzdDVM7$_f+OzkNY@-a9m;<;>Q;xv)sGn*Y|xlW5vJIu9u z&BWBXyk;ji9yPOmax=BRmP@AKgrIAm<-;jL>2r!5XL_&OvFs4<_B(-3=dsJJ{CLDz z|4QZ4-g2p$4=2Ta_x*fm{*8SGBZKcrb3a)I0oRM+yOt~{aBS3R5pd%0@%uM_&c>6H z+l;(tt5$M~{1@>le)dxB{`>z2jAcGHC(Atd_bZsR-{NTIxq@lMbDlGa|D5d|^NcHe z=dsefg2~Z$p7H%JIPma9Qdw@=`Lu7j&km&VE9&`4xvWN2VuWMXDvWn<^yMC+6cQE@6%&_`l#-T_m6KOc zR8m$^Ra4i{)Y8_`)zddHG%_|ZH8Z!cw6eCbwX=6{baHlab#wRd^z!!c_45x13;PSFZ;q=bd{Sgw(_n6Y?AlZfRPhkNIjyLx7b1kY`fRa|D8Th?Z~^o^zH z?WwGP{`1ByXn)uw|7gRm&yJhLl}f4=%%(_hd#!)uS6fg`!OTr16YVP)86GEy-8igx z^z=cUg7&~i_N$%dxG^sd;$!a@U{@A4=GK%<122qB^44iTb3y+*TnxSwcB{TCV_Y=;-0t1de$x|{`xIfoo*co(b z4o`prhZ_G8YY!)h-?AM6jrZC36vDe_c3(W0*d))uv7b=@ScWwSC^0ecG<{)a_78Fv F0RZVK>0kf= literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/data/files/orcfiles/part-r-2.orc b/sql/hive/src/test/resources/data/files/orcfiles/part-r-2.orc new file mode 100755 index 0000000000000000000000000000000000000000..525488226e2a975260e903d3b70e7b39c182cc0c GIT binary patch literal 1286 zcma)+doP*9Az3SGt3A%l#59vj zk!hJBx5%}Kq}j+~jKL7yT(UL{<4$Zba+w_`yMK1}ug`hU^PKm2&ilvfO?A-*0Ih*X zAX(5?0LW5@f`S2L%DA!3ow#ow-GzWLr4xL80a@kJMIb~HF@HThRHgeb2E%d7~b?mMC@jZH1_7~5TS zidGX-w~WJ8t!CVS#p2vtSAl0~aUsWlY9cq!E%V5ei9&ARM?MTU^A8pB>6{btKKMs* zhF47=oqK9YI5gBOpp*;coRf39d5)%6^-A`-S-4jwNZ5`Cm2%seeh2aRLUtBoCU~B7 z_ZXEzxPLyXbuHeqC?HBosEBDly6AEy*qqYe?o+;aEN^7qqKxXd8wEfE69F)rV5zD7 z%WDwWE{Z#z4FS+?HSqoZ^yAWUhuWH&;M4srWf$k|Y>}}?acrXq_Dy%ApOPwDjDC)d zHxq;>loSdgI0Cw0);7B1o30oPsk$|YLVDB@WpS^%J?!kg549J12HBeDP&b*0~u~jHeI5Sg;ez!?@L}7+2Yg|A?OSTM z$lW4yE9lc)=fU{z?g$95SXW}>*jG8+Yw-z*N!M>Ar`$~C-TEypoiE79yq%Sulbd%Z zzu<0RQE^FW*}ePa6%Q(_9#%i9c`W?>Np0QJ`ac>Pn?%jx7D;PcyR@V8Sy%V-7cXDE z?&3ps;9qZthDT(hWAgEd$*Jj?*}3_53yXih|FE>ISoyfRw!Wd%0APD*Jpf)E z6fDacsowoUJEc4+zRh*;vY(X!&UDQ)v+ zqoa#P)6K>ToQVlGi;3_Q&zzFmWPfO7#jYg&GMtUq(L8{&L69-Hhic@VI2(vez}Xmn zZS7Bv1}KI}Ub9#n9!fK94yT2o^>HI3p)g({Aw7SWgYtER_L>ko+rTyyHWvQqR3G;R ze_{9uVu!i4)))d68MPLo3*S;`pEuA Date: Fri, 17 Oct 2014 23:44:57 -0700 Subject: [PATCH 14/18] scala style fixed --- .../org/apache/spark/sql/SQLContext.scala | 6 +- .../apache/spark/sql/hive/HiveContext.scala | 60 +++---------------- .../spark/sql/hive/orc/OrcRelation.scala | 8 +-- .../sql/hive/orc/OrcTableOperations.scala | 6 +- .../apache/spark/sql/hive/orc/package.scala | 2 +- 5 files changed, 15 insertions(+), 67 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 0a0e351c8ac8..23e7b2d27077 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -149,10 +149,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - def parquetFile(path: String): SchemaRDD = { - val a = new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) - a - } + def parquetFile(path: String): SchemaRDD = + new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) /** * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index d1cbe6085376..0f7d5a7fd92b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -23,9 +23,9 @@ import java.util.{ArrayList => JArrayList} import scala.collection.JavaConversions._ import scala.language.implicitConversions -import scala.reflect.runtime.universe.{TypeTag, typeTag} +import scala.reflect.runtime.universe.TypeTag +import scala.Some -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf @@ -39,21 +39,17 @@ import org.apache.hadoop.hive.serde2.io.DateWritable import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.hive.orc.{OrcSchemaRDD, OrcRelation} +import org.apache.spark.sql.hive.orc.OrcSchemaRDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators} import org.apache.spark.sql.catalyst.analysis.{OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.{Command => PhysicalCommand, _} -import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.catalyst.plans.logical.SetCommand -import scala.Some import org.apache.spark.sql.catalyst.plans.logical.NativeCommand -import org.apache.spark.sql.hive.MetastoreRelation import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand -import org.apache.spark.sql.execution.LogicalRDD /** * DEPRECATED: Use HiveContext instead. @@ -116,14 +112,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { @deprecated("hql() is deprecated as the sql function now parses using HiveQL by default. " + s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1") def hql(hqlQuery: String): SchemaRDD = hiveql(hqlQuery) - /** - * Creates a SchemaRDD from an RDD of case classes. - * - * @group userf - */ -// implicit def createOrcSchemaRDD(rdd: SchemaRDD) = { -// new OrcSchemaRDD(rdd) -// } /** * Creates a SchemaRDD from an RDD of case classes. @@ -152,42 +140,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * * @group userf */ - def orcFile(path: String): SchemaRDD = - new SchemaRDD(this, orc.OrcRelation(Seq.empty, path, Some(sparkContext.hadoopConfiguration), this)) - -// /** -// * :: Experimental :: -// * Creates an empty orc file with the schema of class `A`, which can be registered as a table. -// * This registered table can be used as the target of future `insertInto` operations. -// * -// * {{{ -// * val sqlContext = new HiveContext(...) -// * import sqlContext._ -// * -// * case class Person(name: String, age: Int) -// * createOrcFile[Person]("path/to/file.orc").registerTempTable("people") -// * sql("INSERT INTO people SELECT 'michael', 29") -// * }}} -// * -// * @tparam A A case class type that describes the desired schema of the orc file to be -// * created. -// * @param path The path where the directory containing parquet metadata should be created. -// * Data inserted into this table will also be stored at this location. -// * @param allowExisting When false, an exception will be thrown if this directory already exists. -// * @param conf A Hadoop configuration object that can be used to specify options to the parquet -// * output format. -// * -// * @group userf -// */ -// @Experimental -// def createOrcFile[A <: Product : TypeTag]( -// path: String, -// allowExisting: Boolean = true, -// conf: Configuration = new Configuration()): SchemaRDD = { -// new SchemaRDD( -// this, -// OrcRelation.createEmpty(path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) -// } + def orcFile(path: String): SchemaRDD = new SchemaRDD( + this, orc.OrcRelation(Seq.empty, path, Some(sparkContext.hadoopConfiguration), this)) /** * Analyzes the given table in the current database to generate statistics, which will be diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 79d267adb14e..1c453f9ecd2b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -18,24 +18,21 @@ package org.apache.spark.sql.hive.orc import java.util.Properties -import java.io.IOException - import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.ql.io.orc._ -import org.apache.spark.sql.parquet.FileSystemHelper -import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} import org.apache.spark.sql.catalyst.analysis.{UnresolvedException, MultiInstanceRelation} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.hive.HiveMetastoreTypes +import org.apache.spark.sql.parquet.FileSystemHelper +import org.apache.spark.sql.SQLContext import scala.collection.JavaConversions._ -import org.apache.spark.Logging private[sql] case class OrcRelation( attributes: Seq[Attribute], @@ -125,7 +122,6 @@ private[sql] object OrcFileOperator{ val inspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] // data types that is inspected by this inspector val schema = inspector.getTypeName - // set prop here, initial OrcSerde need it val fields = inspector.getAllStructFieldRefs val (columns, columnTypes) = fields.map { f => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala index a036355f6572..0d6655f25760 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala @@ -28,10 +28,11 @@ import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, FileOutputCommi import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.io.{Writable, NullWritable} import org.apache.hadoop.mapreduce.{TaskID, TaskAttemptContext, Job} +import org.apache.hadoop.mapred.{SparkHadoopMapRedUtil, Reporter, JobConf} import org.apache.hadoop.hive.ql.io.orc.{OrcSerde, OrcInputFormat, OrcOutputFormat} import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.ColumnProjectionUtils -import org.apache.hadoop.mapred.{SparkHadoopMapRedUtil, Reporter, JobConf} +import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfoUtils, TypeInfo} import org.apache.spark.sql.execution._ import org.apache.spark.sql.catalyst.expressions._ @@ -39,12 +40,11 @@ import org.apache.spark.sql.parquet.FileSystemHelper import org.apache.spark.{TaskContext, SerializableWritable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode => LogicalUnaryNode} +import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.execution.UnaryNode import org.apache.spark.sql.hive.{HiveMetastoreTypes, HadoopTableReader} import scala.collection.JavaConversions._ -import org.apache.spark.sql.catalyst.types.StructType -import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfoUtils, TypeInfo} /** * logical plan of writing to ORC file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala index 06b2d0e3aee3..cf2a3b87a3df 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala @@ -43,4 +43,4 @@ package object orc { // for prediction push down in hive-0.13.1, don't enable it val ORC_FILTER_PUSHDOWN_ENABLED = false val SARG_PUSHDOWN = "sarg.pushdown" -} \ No newline at end of file +} From b06e33504c30eea973832459a6cc68c85c14b888 Mon Sep 17 00:00:00 2001 From: scwf Date: Fri, 17 Oct 2014 23:49:47 -0700 Subject: [PATCH 15/18] revert no related changes --- .../scala/org/apache/spark/util/Utils.scala | 12 +++---- docs/sql-programming-guide.md | 33 ------------------- 2 files changed, 4 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 03a484a6cb9d..53a7512edd85 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -659,15 +659,11 @@ private[spark] object Utils extends Logging { } private def listFilesSafely(file: File): Seq[File] = { - if (file.exists()) { - val files = file.listFiles() - if (files == null) { - throw new IOException("Failed to list files for dir: " + file) - } - files - } else { - List() + val files = file.listFiles() + if (files == null) { + throw new IOException("Failed to list files for dir: " + file) } + files } /** diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 368acf68fa6a..233fe1225a43 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -687,39 +687,6 @@ for teenName in teenNames.collect():
-### Configuration - -Configuration of ORC can be done using the `setConf` method on HiveContext or by running -`SET key=value` commands using SQL. - - - - - - - - - - - - - - - - - - -
Property NameDefaultMeaning
spark.sql.parquet.binaryAsStringfalse - Some other Parquet-producing systems, in particular Impala and older versions of Spark SQL, do - not differentiate between binary data and strings when writing out the Parquet schema. This - flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. -
spark.sql.parquet.cacheMetadatafalse - Turns on caching of Parquet schema metadata. Can speed up querying of static data. -
spark.sql.parquet.compression.codecsnappy - Sets the compression codec use when writing Parquet files. Acceptable values include: - uncompressed, snappy, gzip, lzo. -
- ## JSON Datasets
From 4d0950b198027850a9caf0bb292fad570d87cf5c Mon Sep 17 00:00:00 2001 From: scwf Date: Sat, 18 Oct 2014 01:14:37 -0700 Subject: [PATCH 16/18] more tests --- .../sql/parquet/ParquetTableOperations.scala | 3 +- .../spark/sql/hive/orc/ORCQuerySuite.scala | 146 +++++++++++++++++- 2 files changed, 145 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index e3069e477295..e52a5eea7a44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -497,7 +497,7 @@ private[parquet] object FilteringParquetRowInputFormat { .build[FileStatus, Array[BlockLocation]]() } -private[sql] object FileSystemHelper extends Logging{ +private[sql] object FileSystemHelper { def listFiles(pathStr: String, conf: Configuration): Seq[Path] = { val origPath = new Path(pathStr) val fs = origPath.getFileSystem(conf) @@ -526,7 +526,6 @@ private[sql] object FileSystemHelper extends Logging{ if (fs.exists(path)) { fs.listStatus(path).map(_.getPath).filter(p => p.getName.endsWith(extension)) } else { - println(s"${path.toString} not exists") Seq.empty } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala index 0d883bb9b400..c439ef532d0f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala @@ -28,9 +28,32 @@ import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.util.Utils import java.io.File +import org.apache.hadoop.hive.ql.io.orc.CompressionKind case class TestRDDEntry(key: Int, value: String) +case class NullReflectData( + intField: java.lang.Integer, + longField: java.lang.Long, + floatField: java.lang.Float, + doubleField: java.lang.Double, + booleanField: java.lang.Boolean) + +case class OptionalReflectData( + intField: Option[Int], + longField: Option[Long], + floatField: Option[Float], + doubleField: Option[Double], + booleanField: Option[Boolean]) + +case class Nested(i: Int, s: String) + +case class Data(array: Seq[Int], nested: Nested) + +case class Contact(name: String, phone: String) + +case class Person(name: String, age: Int, contacts: Seq[Contact]) + case class AllDataTypes( stringField: String, intField: Int, @@ -41,13 +64,32 @@ case class AllDataTypes( byteField: Byte, booleanField: Boolean) +case class AllDataTypesWithNonPrimitiveType( + stringField: String, + intField: Int, + longField: Long, + floatField: Float, + doubleField: Double, + shortField: Short, + byteField: Byte, + booleanField: Boolean, + array: Seq[Int], + arrayContainsNull: Seq[Option[Int]], + map: Map[Int, Long], + mapValueContainsNull: Map[Int, Option[Long]], + data: Data) + +case class BinaryData(binaryData: Array[Byte]) + class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { test("Read/Write All Types") { val tempDir = getTempFilePath("orcTest").getCanonicalPath val range = (0 to 255) - val data = sparkContext.parallelize(range) - .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + val data = sparkContext + .parallelize(range) + .map(x => AllDataTypes( + s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) data.saveAsOrcFile(tempDir) checkAnswer( @@ -57,6 +99,79 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { Utils.deleteRecursively(new File(tempDir)) } + test("read/write binary data") { + val tempDir = getTempFilePath("orcTest").getCanonicalPath + sparkContext.parallelize(BinaryData("test".getBytes("utf8")) :: Nil).saveAsOrcFile(tempDir) + TestHive.orcFile(tempDir) + .map(r => new String(r(0).asInstanceOf[Array[Byte]], "utf8")) + .collect().toSeq == Seq("test") + Utils.deleteRecursively(new File(tempDir)) + } + + test("Read/Write All Types with non-primitive type") { + val tempDir = getTempFilePath("orcTest").getCanonicalPath + val range = (0 to 255) + val data = sparkContext.parallelize(range) + .map(x => AllDataTypesWithNonPrimitiveType( + s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, + (0 until x), + (0 until x).map(Option(_).filter(_ % 3 == 0)), + (0 until x).map(i => i -> i.toLong).toMap, + (0 until x).map(i => i -> Option(i.toLong)).toMap + (x -> None), + Data((0 until x), Nested(x, s"$x")))) + data.saveAsOrcFile(tempDir) + + checkAnswer( + TestHive.orcFile(tempDir), + data.toSchemaRDD.collect().toSeq) + Utils.deleteRecursively(new File(tempDir)) + } + + test("Creating case class RDD table") { + sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + .registerTempTable("tmp") + val rdd = sql("SELECT * FROM tmp").collect().sortBy(_.getInt(0)) + var counter = 1 + rdd.foreach { + // '===' does not like string comparison? + row => { + assert(row.getString(1).equals(s"val_$counter"), s"row $counter value ${row.getString(1)} does not match val_$counter") + counter = counter + 1 + } + } + } + + test("Simple selection form orc table") { + val tempDir = getTempFilePath("orcTest").getCanonicalPath + val data = sparkContext.parallelize((1 to 10)) + .map(i => Person(s"name_$i", i, (0 until 2).map{ m=> + Contact(s"contact_$m", s"phone_$m") })) + data.saveAsOrcFile(tempDir) + val f = TestHive.orcFile(tempDir) + f.registerTempTable("tmp") + var rdd = sql("SELECT name FROM tmp where age <= 5") + assert(rdd.count() == 5) + + rdd = sql("SELECT name, contacts FROM tmp where age > 5") + assert(rdd.count() == 5) + + val contacts = rdd.flatMap(t=>t(1).asInstanceOf[Seq[_]]) + assert(contacts.count() == 10) + Utils.deleteRecursively(new File(tempDir)) + } + + test("save and load case class RDD with Nones as orc") { + val data = OptionalReflectData(None, None, None, None, None) + val rdd = sparkContext.parallelize(data :: Nil) + val tempDir = getTempFilePath("orcTest").getCanonicalPath + rdd.saveAsOrcFile(tempDir) + val readFile = TestHive.orcFile(tempDir) + val rdd_saved = readFile.collect() + assert(rdd_saved(0) === Seq.fill(5)(null)) + Utils.deleteRecursively(new File(tempDir)) + } + test("Compression options for writing to a Orcfile") { val tempDir = getTempFilePath("orcTest").getCanonicalPath val rdd = TestHive.sparkContext.parallelize((1 to 100)) @@ -82,4 +197,31 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { assert(attributes(6).dataType == ByteType) assert(attributes(7).dataType == BooleanType) } + + ignore("Other Compression options for writing to an Orcfile only supported in hive 0.13.1 and above") { + TestHive.sparkContext.hadoopConfiguration.set(orcDefaultCompressVar, "SNAPPY") + var tempDir = getTempFilePath("orcTest").getCanonicalPath + val rdd = sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + rdd.saveAsOrcFile(tempDir) + var actualCodec = OrcFileOperator.getMetaDataReader(new Path(tempDir), Some(new Configuration())).getCompression + assert(actualCodec == CompressionKind.SNAPPY) + Utils.deleteRecursively(new File(tempDir)) + + TestHive.sparkContext.hadoopConfiguration.set(orcDefaultCompressVar, "NONE") + tempDir = getTempFilePath("orcTest").getCanonicalPath + rdd.saveAsOrcFile(tempDir) + actualCodec = OrcFileOperator.getMetaDataReader(new Path(tempDir), Some(new Configuration())).getCompression + assert(actualCodec == CompressionKind.NONE) + Utils.deleteRecursively(new File(tempDir)) + + TestHive.sparkContext.hadoopConfiguration.set(orcDefaultCompressVar, "LZO") + tempDir = getTempFilePath("orcTest").getCanonicalPath + rdd.saveAsOrcFile(tempDir) + actualCodec = OrcFileOperator.getMetaDataReader(new Path(tempDir), Some(new Configuration())).getCompression + assert(actualCodec == CompressionKind.LZO) + Utils.deleteRecursively(new File(tempDir)) + } + + } From 37a21927b2c5a8e633fc4c727542f639e7a52569 Mon Sep 17 00:00:00 2001 From: scwf Date: Sat, 18 Oct 2014 03:37:33 -0700 Subject: [PATCH 17/18] tests suite fix --- .../sql/hive/orc/OrcTableOperations.scala | 2 +- .../apache/spark/sql/hive/orc/package.scala | 37 ++++++++++++++++++- .../spark/sql/hive/orc/ORCQuerySuite.scala | 15 ++++---- 3 files changed, 45 insertions(+), 9 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala index 0d6655f25760..f15a7c5a4a81 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala @@ -202,7 +202,7 @@ private[sql] case class InsertIntoOrcTable( iter.map { row => var i = 0 while (i < row.length) { - outputData(i) = HadoopTableReader.unwrapData(row(i), fieldOIs(i)) + outputData(i) = HadoopTypeConverter.wrap((row(i), fieldOIs(i))) i += 1 } orcSerde.serialize(outputData, standardOI) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala index cf2a3b87a3df..7998e6a60f07 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala @@ -19,7 +19,12 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.{SQLContext, SchemaRDD} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{JavaHiveDecimalObjectInspector, JavaHiveVarcharObjectInspector} +import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} +import org.apache.spark.sql.catalyst.expressions.Row +/* Implicit conversions */ +import scala.collection.JavaConversions._ package object orc { class OrcSchemaRDD( @transient val sqlContext1: SQLContext, @@ -38,6 +43,36 @@ package object orc { } } + object HadoopTypeConverter extends HiveInspectors { + def wrap(a: (Any, ObjectInspector)): Any = a match { + case (s: String, oi: JavaHiveVarcharObjectInspector) => + new HiveVarchar(s, s.size) + + case (bd: BigDecimal, oi: JavaHiveDecimalObjectInspector) => + new HiveDecimal(bd.underlying()) + + case (row: Row, oi: StandardStructObjectInspector) => + val struct = oi.create() + row.zip(oi.getAllStructFieldRefs: Seq[StructField]).foreach { + case (data, field) => + oi.setStructFieldData(struct, field, wrap(data, field.getFieldObjectInspector)) + } + struct + case (s: Seq[_], oi: ListObjectInspector) => + val wrappedSeq = s.map(wrap(_, oi.getListElementObjectInspector)) + seqAsJavaList(wrappedSeq) + + case (m: Map[_, _], oi: MapObjectInspector) => + val keyOi = oi.getMapKeyObjectInspector + val valueOi = oi.getMapValueObjectInspector + val wrappedMap = m.map { case (key, value) => wrap(key, keyOi) -> wrap(value, valueOi) } + mapAsJavaMap(wrappedMap) + + case (obj, _) => + obj + } + } + // for orc compression type, only take effect in hive 0.13.1 val orcDefaultCompressVar = "hive.exec.orc.default.compress" // for prediction push down in hive-0.13.1, don't enable it diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala index c439ef532d0f..791d7c01066d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/ORCQuerySuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.orc import java.util.Properties +import java.io.File import org.scalatest.BeforeAndAfterAll import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -27,7 +28,6 @@ import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.util.Utils -import java.io.File import org.apache.hadoop.hive.ql.io.orc.CompressionKind case class TestRDDEntry(key: Int, value: String) @@ -101,10 +101,13 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { test("read/write binary data") { val tempDir = getTempFilePath("orcTest").getCanonicalPath - sparkContext.parallelize(BinaryData("test".getBytes("utf8")) :: Nil).saveAsOrcFile(tempDir) + val range = (0 to 3) + sparkContext.parallelize(range) + .map(x => BinaryData(s"test$x".getBytes("utf8"))).saveAsOrcFile(tempDir) + TestHive.orcFile(tempDir) .map(r => new String(r(0).asInstanceOf[Array[Byte]], "utf8")) - .collect().toSeq == Seq("test") + .collect().toSet == Set("test0", "test1", "test2", "test3") Utils.deleteRecursively(new File(tempDir)) } @@ -163,7 +166,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { test("save and load case class RDD with Nones as orc") { val data = OptionalReflectData(None, None, None, None, None) - val rdd = sparkContext.parallelize(data :: Nil) + val rdd = sparkContext.parallelize(data :: data :: data :: Nil) val tempDir = getTempFilePath("orcTest").getCanonicalPath rdd.saveAsOrcFile(tempDir) val readFile = TestHive.orcFile(tempDir) @@ -186,7 +189,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { } test("Get ORC Schema with ORC Reader") { - val path = "sql/hive/src/test/resources/data/files/orcfiles" + val path = "src/test/resources/data/files/orcfiles" val attributes = OrcFileOperator.orcSchema(path, Some(TestHive.sparkContext.hadoopConfiguration), new Properties()) assert(attributes(0).dataType == StringType) assert(attributes(1).dataType == IntegerType) @@ -222,6 +225,4 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll { assert(actualCodec == CompressionKind.LZO) Utils.deleteRecursively(new File(tempDir)) } - - } From 4e1c839960c6ca25ec16d88a750f75a28a25d4bf Mon Sep 17 00:00:00 2001 From: scwf Date: Sat, 18 Oct 2014 21:35:57 -0700 Subject: [PATCH 18/18] minor fix --- .../src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 2 +- .../org/apache/spark/sql/hive/orc/OrcTableOperations.scala | 2 +- .../src/main/scala/org/apache/spark/sql/hive/orc/package.scala | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 0f7d5a7fd92b..8d6530d30845 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -23,7 +23,7 @@ import java.util.{ArrayList => JArrayList} import scala.collection.JavaConversions._ import scala.language.implicitConversions -import scala.reflect.runtime.universe.TypeTag +import scala.reflect.runtime.universe.{TypeTag, typeTag} import scala.Some import org.apache.hadoop.fs.FileSystem diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala index f15a7c5a4a81..82aeb12a55a8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala @@ -132,7 +132,7 @@ case class OrcTableScan( * @return Pruned TableScan. */ def pruneColumns(prunedAttributes: Seq[Attribute]): OrcTableScan = { - // Todo: prune projection + // TODO: prune projection OrcTableScan(prunedAttributes, relation, columnPruningPred) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala index 7998e6a60f07..14fe51b92d4b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/package.scala @@ -43,6 +43,7 @@ package object orc { } } + // TypeConverter for InsertIntoOrcTable object HadoopTypeConverter extends HiveInspectors { def wrap(a: (Any, ObjectInspector)): Any = a match { case (s: String, oi: JavaHiveVarcharObjectInspector) =>